You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/04/22 23:20:46 UTC

[GitHub] [arrow] anthonylouisbsb opened a new pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions - WIP

anthonylouisbsb opened a new pull request #10137:
URL: https://github.com/apache/arrow/pull/10137


   


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

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



[GitHub] [arrow] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,473 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response =
+      cast_interval_day(&execution_context_, data.data(), 8, true, &out_valid);
+  int64_t qty_days_in_response = 0;
+  int64_t qty_millis_in_response = 73834992;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data.data(), 8, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 73834992);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";

Review comment:
       maybe 1 test case with decimal in the final as per rfc?
   e.f P12Y15,5D (if that is legal)




-- 
This is an automated message from the 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 #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions - WIP

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


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


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

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



[GitHub] [arrow] anthonylouisbsb commented on pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   @projjal [This thread in the date.h library](https://github.com/HowardHinnant/date/issues/666) updated. The values for interval still need to be parsed using `regex`, because it is not implemented by the library.


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

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



[GitHub] [arrow] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       Done




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

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

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



[GitHub] [arrow] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass years and days and months to cast
+  data = "P12Y2M15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass days and months to cast
+  data = "P5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass all possible fields cast
+  data = "P2Y5M13DT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields cast
+  data = "PT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields to cast without hours
+  data = "PT42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only weeks to cast
+  data = "P25W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);

Review comment:
       I based on some engines behaviors(like Dremio as example) where the period in week format is not compatible to return the month interval.




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;

Review comment:
       I changed the variable name




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;

Review comment:
       I put the value in a static const variable




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass years and days and months to cast
+  data = "P12Y2M15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass days and months to cast
+  data = "P5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass all possible fields cast
+  data = "P2Y5M13DT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields cast
+  data = "PT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields to cast without hours
+  data = "PT42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only weeks to cast
+  data = "P25W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestMatchErrorsForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass an empty string
+  bool out_valid;
+  std::string data(" ");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass only days before years
+  data = "P15D12Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass years and days and months in wrong order
+  data = "P12M15D2Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Forget the P in the first position
+  data = "5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Use m instead M in the period format
+  data = "P2Y5M13DT10H42m21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Does not pass the T when defining only time fields
+  data = "P10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass weeks with other variables
+  data = "P2Y25W2M3D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingWeekFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("P1W");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 7);
+
+  data = "P10W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 70);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("1742461111");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  int64_t qty_days_in_response = 20;
+  int64_t qty_millis_in_response = 14461111;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P1Y1M1DT1H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 1;
+  qty_millis_in_response = 3661000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT48H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 2;
+  qty_millis_in_response = 61000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P10DT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 10;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+
+  data = "P0DT0S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 0;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalYear) {
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_years = *interval_years_holder;
+
+  bool out_valid;
+  std::string data("65851111");

Review comment:
       The comments were wrong, the input is the number of months. I fixed the comment in the function.




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

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

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



[GitHub] [arrow] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/gdv_function_stubs.cc
##########
@@ -794,6 +795,53 @@ const char* gdv_fn_initcap_utf8(int64_t context, const char* data, int32_t data_
   *out_len = out_idx;
   return out;
 }
+
+int64_t gdv_fn_cast_intervalday_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                     const char* data, int data_len, bool in1_validity,
+                                     bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int64_t gdv_fn_cast_intervalday_utf8_int32(int64_t context_ptr, int64_t holder_ptr,
+                                           const char* data, int data_len,
+                                           bool in1_validity, int32_t /*suppress_errors*/,
+                                           bool /*in3_validity*/, bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int32_t gdv_fn_cast_intervalyear_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                      const char* data, int data_len, bool in1_validity,
+                                      bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalYearsHolder* holder =
+      reinterpret_cast<gandiva::IntervalYearsHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);

Review comment:
       It is because there is a moment that I need to pass the value to the regex matcher, e.g: https://github.com/apache/arrow/blob/637c60eaa40cb6ce3a11813e5c0eee2096721f4c/cpp/src/gandiva/interval_holder.cc#L157
   
   And if I pass the pointer, It throws an error as the buffer contains all strings concatenated.




-- 
This is an automated message from the 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] pravindra closed pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   


-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       I put the value in a static const variable




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (total_days & 0x00000000FFFFFFFF);
+  out |= ((module_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+// The operator will cast a generic string defined by the user into an interval of days.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int64_t IntervalDaysHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                       bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalDayFromMillis(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    return GetIntervalDayFromWeeks(period_in_weeks, out_valid);
+  }
+
+  std::string days_in_period;
+  std::string hours_in_period;
+  std::string minutes_in_period;
+  std::string seconds_in_period;
+  std::string ignored_string;  // string to store unnecessary captured groups
+  if (std::regex_match(data, period_not_contains_time)) {
+    if (RE2::FullMatch(data, iso8601_period_without_time, &ignored_string,
+                       &ignored_string, &days_in_period)) {
+      return GetIntervalDayFromCompletePeriod(days_in_period, hours_in_period,
+                                              minutes_in_period, seconds_in_period,
+                                              out_valid);
+    }
+
+    return_error(ctx, data);
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, iso8601_complete_period, &ignored_string, &ignored_string,
+                     &days_in_period, &hours_in_period, &minutes_in_period,
+                     &seconds_in_period)) {
+    return GetIntervalDayFromCompletePeriod(
+        days_in_period, hours_in_period, minutes_in_period, seconds_in_period, out_valid);
+  }
+
+  return_error(ctx, data);
+  return 0;
+}
+
+Status IntervalDaysHolder::Make(const FunctionNode& node,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  const std::string function_name("castINTERVALDAY");
+  return IntervalHolder<IntervalDaysHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalDaysHolder::Make(int32_t suppress_errors,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  return IntervalHolder<IntervalDaysHolder>::Make(suppress_errors, holder);
+}
+
+Status IntervalYearsHolder::Make(const FunctionNode& node,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  const std::string function_name("castINTERVALYEAR");
+  return IntervalHolder<IntervalYearsHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalYearsHolder::Make(int32_t suppress_errors,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  return IntervalHolder<IntervalYearsHolder>::Make(suppress_errors, holder);
+}
+
+// The operator will cast a generic string defined by the user into an interval of months.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int32_t IntervalYearsHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                        bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalYearFromNumber(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    *out_valid = true;

Review comment:
       The reason for the existence of that branch is because [in ISO](https://en.wikipedia.org/wiki/ISO_8601#Durations) the user must choose to represent a period using one of these two ways:
   
   - `PnYnMTnHnM`
   - `PnW`
   
   So if the user represents using the second format, we will process the value in a different 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);

Review comment:
       I changed to match an exact 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] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));

Review comment:
       ok. in that case shouldn't you be doing floating point arithmetic and only cast the final qty_millis as int64_t. Otherwise, wouldn't you miss some days or millis.
   
   e,g consider PW30.5
   
   in the above stod returns 30.5. but you cast it to int64_t, which will cause the fractionary part to be lost. Or am I missing something? instead if you use double 
     auto qty_weeks = std::stod(...)
     int64_t millis_in_all_weeks = static_cast<int64_t> (qty_weeks * kMillisInAWeek);
     
   you will not loose nearly 1/2 a week.




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));

Review comment:
       The answer is similar to that question: https://github.com/apache/arrow/pull/10137#discussion_r756488410




-- 
This is an automated message from the 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] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass years and days and months to cast
+  data = "P12Y2M15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass days and months to cast
+  data = "P5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass all possible fields cast
+  data = "P2Y5M13DT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields cast
+  data = "PT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields to cast without hours
+  data = "PT42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only weeks to cast
+  data = "P25W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestMatchErrorsForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass an empty string
+  bool out_valid;
+  std::string data(" ");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass only days before years
+  data = "P15D12Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass years and days and months in wrong order
+  data = "P12M15D2Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Forget the P in the first position
+  data = "5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Use m instead M in the period format
+  data = "P2Y5M13DT10H42m21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Does not pass the T when defining only time fields
+  data = "P10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass weeks with other variables
+  data = "P2Y25W2M3D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingWeekFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("P1W");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 7);
+
+  data = "P10W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 70);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("1742461111");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  int64_t qty_days_in_response = 20;
+  int64_t qty_millis_in_response = 14461111;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P1Y1M1DT1H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 1;
+  qty_millis_in_response = 3661000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT48H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 2;
+  qty_millis_in_response = 61000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P10DT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 10;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+
+  data = "P0DT0S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 0;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalYear) {
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_years = *interval_years_holder;
+
+  bool out_valid;
+  std::string data("65851111");

Review comment:
       ok




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

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 #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   Benchmark runs are scheduled for baseline = 40ac81087a9d6d97dae4b11c28a147176d22dc54 and contender = adfb91302231418c5de53d427ddb7e8c3b6abee1. adfb91302231418c5de53d427ddb7e8c3b6abee1 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/c90fbedfe860480e921ccb7ef9a52257...ea88e1db3ed54ace950a544460b33c93/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/cce9eda123e64b89a3bc11f51eb8fbad...0aaddd68a6a5461489eaf1b42f7ed356/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/9f501bcda19f44a096c12bd9990a15c0...ebeb4a2ccbc9400ab9c4508a615bf39a/)
   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] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/gdv_function_stubs.cc
##########
@@ -794,6 +795,53 @@ const char* gdv_fn_initcap_utf8(int64_t context, const char* data, int32_t data_
   *out_len = out_idx;
   return out;
 }
+
+int64_t gdv_fn_cast_intervalday_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                     const char* data, int data_len, bool in1_validity,
+                                     bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int64_t gdv_fn_cast_intervalday_utf8_int32(int64_t context_ptr, int64_t holder_ptr,
+                                           const char* data, int data_len,
+                                           bool in1_validity, int32_t /*suppress_errors*/,
+                                           bool /*in3_validity*/, bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int32_t gdv_fn_cast_intervalyear_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                      const char* data, int data_len, bool in1_validity,
+                                      bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalYearsHolder* holder =
+      reinterpret_cast<gandiva::IntervalYearsHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);

Review comment:
       Isn't this expensive as std::string does an allocate and a deep copy and then a free? not sure if it easy to change the interface to take something other than a string, though 

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (total_days & 0x00000000FFFFFFFF);
+  out |= ((module_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+// The operator will cast a generic string defined by the user into an interval of days.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int64_t IntervalDaysHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                       bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalDayFromMillis(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    return GetIntervalDayFromWeeks(period_in_weeks, out_valid);
+  }
+
+  std::string days_in_period;
+  std::string hours_in_period;
+  std::string minutes_in_period;
+  std::string seconds_in_period;
+  std::string ignored_string;  // string to store unnecessary captured groups
+  if (std::regex_match(data, period_not_contains_time)) {
+    if (RE2::FullMatch(data, iso8601_period_without_time, &ignored_string,
+                       &ignored_string, &days_in_period)) {
+      return GetIntervalDayFromCompletePeriod(days_in_period, hours_in_period,
+                                              minutes_in_period, seconds_in_period,
+                                              out_valid);
+    }
+
+    return_error(ctx, data);
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, iso8601_complete_period, &ignored_string, &ignored_string,
+                     &days_in_period, &hours_in_period, &minutes_in_period,
+                     &seconds_in_period)) {
+    return GetIntervalDayFromCompletePeriod(
+        days_in_period, hours_in_period, minutes_in_period, seconds_in_period, out_valid);
+  }
+
+  return_error(ctx, data);
+  return 0;
+}
+
+Status IntervalDaysHolder::Make(const FunctionNode& node,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  const std::string function_name("castINTERVALDAY");
+  return IntervalHolder<IntervalDaysHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalDaysHolder::Make(int32_t suppress_errors,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  return IntervalHolder<IntervalDaysHolder>::Make(suppress_errors, holder);
+}
+
+Status IntervalYearsHolder::Make(const FunctionNode& node,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  const std::string function_name("castINTERVALYEAR");
+  return IntervalHolder<IntervalYearsHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalYearsHolder::Make(int32_t suppress_errors,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  return IntervalHolder<IntervalYearsHolder>::Make(suppress_errors, holder);
+}
+
+// The operator will cast a generic string defined by the user into an interval of months.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int32_t IntervalYearsHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                        bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalYearFromNumber(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    *out_valid = true;

Review comment:
       did not understand this branch.  If we specify period in weeks (say > 52 weeks) won't it correspond to several months? or is this illegal..if it is illegal. shouln't we be returning an error?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       same as above

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       can this number be defined in a common place as MILLIS_IN_A_DAY either as a static const or a #define?

##########
File path: cpp/src/gandiva/gdv_function_stubs.cc
##########
@@ -794,6 +795,53 @@ const char* gdv_fn_initcap_utf8(int64_t context, const char* data, int32_t data_
   *out_len = out_idx;
   return out;
 }
+
+int64_t gdv_fn_cast_intervalday_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                     const char* data, int data_len, bool in1_validity,
+                                     bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int64_t gdv_fn_cast_intervalday_utf8_int32(int64_t context_ptr, int64_t holder_ptr,
+                                           const char* data, int data_len,
+                                           bool in1_validity, int32_t /*suppress_errors*/,
+                                           bool /*in3_validity*/, bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);

Review comment:
       can data_len be zero or less? if so, what happens to std::string construction?

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);

Review comment:
       why can't we exactly assert for the expected days and millis, by splitting the response to two 32 bits?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour

Review comment:
       feel it is more readable if a pre-processor macro or a static const is used, e,g MILLIS_IN_AN_HOUR and so on..

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;

Review comment:
       this is not millis_in_a_week, isn't it total_millis_in_all_weeks?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;

Review comment:
       did you mean modulo_millis?. module_millis is a confusing name. maybe remainder_millis or something like that?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;

Review comment:
       Can't millis in a week be pre-computed in the pre-processor stage,  either as a static const, e.g kMillisInAWeek = kMillisInADay * 7 or as a macro?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));

Review comment:
       here a static cast on stod is used for qty_days, but an stol is used for qty_days in a previous method. any reason for this?

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);

Review comment:
       please assert for exact expected value. similar comment for all EXPECT_GT below..

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (total_days & 0x00000000FFFFFFFF);
+  out |= ((module_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+// The operator will cast a generic string defined by the user into an interval of days.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int64_t IntervalDaysHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                       bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalDayFromMillis(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    return GetIntervalDayFromWeeks(period_in_weeks, out_valid);
+  }
+
+  std::string days_in_period;
+  std::string hours_in_period;
+  std::string minutes_in_period;
+  std::string seconds_in_period;
+  std::string ignored_string;  // string to store unnecessary captured groups
+  if (std::regex_match(data, period_not_contains_time)) {
+    if (RE2::FullMatch(data, iso8601_period_without_time, &ignored_string,
+                       &ignored_string, &days_in_period)) {
+      return GetIntervalDayFromCompletePeriod(days_in_period, hours_in_period,
+                                              minutes_in_period, seconds_in_period,
+                                              out_valid);
+    }
+
+    return_error(ctx, data);
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, iso8601_complete_period, &ignored_string, &ignored_string,
+                     &days_in_period, &hours_in_period, &minutes_in_period,
+                     &seconds_in_period)) {
+    return GetIntervalDayFromCompletePeriod(
+        days_in_period, hours_in_period, minutes_in_period, seconds_in_period, out_valid);
+  }
+
+  return_error(ctx, data);
+  return 0;
+}
+
+Status IntervalDaysHolder::Make(const FunctionNode& node,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  const std::string function_name("castINTERVALDAY");
+  return IntervalHolder<IntervalDaysHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalDaysHolder::Make(int32_t suppress_errors,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  return IntervalHolder<IntervalDaysHolder>::Make(suppress_errors, holder);
+}
+
+Status IntervalYearsHolder::Make(const FunctionNode& node,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  const std::string function_name("castINTERVALYEAR");
+  return IntervalHolder<IntervalYearsHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalYearsHolder::Make(int32_t suppress_errors,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  return IntervalHolder<IntervalYearsHolder>::Make(suppress_errors, holder);
+}
+
+// The operator will cast a generic string defined by the user into an interval of months.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int32_t IntervalYearsHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                        bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalYearFromNumber(data, out_valid);

Review comment:
       the comment above says this is a period in millis that is converted to months..but GetIntervalYearFromNumber is reading it directly as number of months. either you have to fix the comments or the code, no?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));

Review comment:
       any reason why stod is used here, whereas stol is used in the previous. Is it because weeks can be specified in partial weeks (e.g 2.5). If that is the case, will doing just a static_cast loose some information?

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);

Review comment:
       I think we should assert for exact expected value

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass years and days and months to cast
+  data = "P12Y2M15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass days and months to cast
+  data = "P5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass all possible fields cast
+  data = "P2Y5M13DT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields cast
+  data = "PT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields to cast without hours
+  data = "PT42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only weeks to cast
+  data = "P25W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestMatchErrorsForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass an empty string
+  bool out_valid;
+  std::string data(" ");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass only days before years
+  data = "P15D12Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass years and days and months in wrong order
+  data = "P12M15D2Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Forget the P in the first position
+  data = "5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Use m instead M in the period format
+  data = "P2Y5M13DT10H42m21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Does not pass the T when defining only time fields
+  data = "P10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass weeks with other variables
+  data = "P2Y25W2M3D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingWeekFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("P1W");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 7);
+
+  data = "P10W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 70);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("1742461111");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  int64_t qty_days_in_response = 20;
+  int64_t qty_millis_in_response = 14461111;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P1Y1M1DT1H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 1;
+  qty_millis_in_response = 3661000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT48H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 2;
+  qty_millis_in_response = 61000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P10DT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 10;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+
+  data = "P0DT0S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 0;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalYear) {
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_years = *interval_years_holder;
+
+  bool out_valid;
+  std::string data("65851111");

Review comment:
       this looks a bit strange and confusing. If we are expecting input in months (and not millis), the comments in the operator above should be corrected. Also, if we expect months, something smaller like 623 months etc might be better as this looks like a large value for months.

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       millis in a day seems to be occuring more than once, so surely a pre-processor or a static const might be better

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass years and days and months to cast
+  data = "P12Y2M15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass days and months to cast
+  data = "P5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass all possible fields cast
+  data = "P2Y5M13DT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields cast
+  data = "PT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields to cast without hours
+  data = "PT42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only weeks to cast
+  data = "P25W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);

Review comment:
       can this really be zero, for me 25 weeks is around 5 months.? if this is an error or if one is not expected to do castIntervalYear on a week format, what is expected?




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

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

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



[GitHub] [arrow] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));

Review comment:
       ok. in that case shouldn't you be doing floating point arithmetic and only cast the final qty_millis as double. Otherwise, wouldn't you miss some days or millis.
   
   e,g consider PW30.5
   
   in the above stod returns 30.5. but you cast it to int64_t, which will cause the fractionary part to be lost. Or am I missing something? instead if you use double 
     auto qty_weeks = std::stod(...)
     int64_t millis_in_all_weeks = static_cast<int64_t> (qty_weeks * kMillisInAWeek);
     
   you will not loose nearly 1/2 a week.




-- 
This is an automated message from the 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] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (total_days & 0x00000000FFFFFFFF);
+  out |= ((module_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+// The operator will cast a generic string defined by the user into an interval of days.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int64_t IntervalDaysHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                       bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalDayFromMillis(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    return GetIntervalDayFromWeeks(period_in_weeks, out_valid);
+  }
+
+  std::string days_in_period;
+  std::string hours_in_period;
+  std::string minutes_in_period;
+  std::string seconds_in_period;
+  std::string ignored_string;  // string to store unnecessary captured groups
+  if (std::regex_match(data, period_not_contains_time)) {
+    if (RE2::FullMatch(data, iso8601_period_without_time, &ignored_string,
+                       &ignored_string, &days_in_period)) {
+      return GetIntervalDayFromCompletePeriod(days_in_period, hours_in_period,
+                                              minutes_in_period, seconds_in_period,
+                                              out_valid);
+    }
+
+    return_error(ctx, data);
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, iso8601_complete_period, &ignored_string, &ignored_string,
+                     &days_in_period, &hours_in_period, &minutes_in_period,
+                     &seconds_in_period)) {
+    return GetIntervalDayFromCompletePeriod(
+        days_in_period, hours_in_period, minutes_in_period, seconds_in_period, out_valid);
+  }
+
+  return_error(ctx, data);
+  return 0;
+}
+
+Status IntervalDaysHolder::Make(const FunctionNode& node,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  const std::string function_name("castINTERVALDAY");
+  return IntervalHolder<IntervalDaysHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalDaysHolder::Make(int32_t suppress_errors,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  return IntervalHolder<IntervalDaysHolder>::Make(suppress_errors, holder);
+}
+
+Status IntervalYearsHolder::Make(const FunctionNode& node,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  const std::string function_name("castINTERVALYEAR");
+  return IntervalHolder<IntervalYearsHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalYearsHolder::Make(int32_t suppress_errors,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  return IntervalHolder<IntervalYearsHolder>::Make(suppress_errors, holder);
+}
+
+// The operator will cast a generic string defined by the user into an interval of months.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int32_t IntervalYearsHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                        bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalYearFromNumber(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    *out_valid = true;

Review comment:
       and are you saying that if we specify period in weeks, the number of months should be always 0, as per rfc. In that case I understand this branch 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] anthonylouisbsb commented on pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   @rkavanap I added the test cases for float 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] anthonylouisbsb commented on pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   @jpedroantunes could you review the PR, please?


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

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



[GitHub] [arrow] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;

Review comment:
       Changed the name to `remainder_millis`, thanks for the suggestion.




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));

Review comment:
       The correct way is to use `stod` in all places because the ISO format allows the user to pass decimal numbers https://en.wikipedia.org/wiki/ISO_8601#Time_intervals.
   
   The reason that I cast as long is that I based on Joda Time implementation: https://www.joda.org/joda-time/apidocs/org/joda/time/Period.html that was the base of Java 8 Period API and that API returns only the integer part of the values.




-- 
This is an automated message from the 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] anthonylouisbsb commented on pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   @projjal I want to move the logic to convert the period to interval_day to the **arrow/util/value_parsing.h** header. But I checked that the files inside the **arrow/util** folder do not use external dependencies, so I did not found any place in CmakeLists.txt to add the external libraries and some methods depend on the RE2 library to 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.

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



[GitHub] [arrow] ursabot edited a comment on pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   Benchmark runs are scheduled for baseline = 40ac81087a9d6d97dae4b11c28a147176d22dc54 and contender = adfb91302231418c5de53d427ddb7e8c3b6abee1. adfb91302231418c5de53d427ddb7e8c3b6abee1 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/c90fbedfe860480e921ccb7ef9a52257...ea88e1db3ed54ace950a544460b33c93/)
   [Failed :arrow_down:0.36% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/cce9eda123e64b89a3bc11f51eb8fbad...0aaddd68a6a5461489eaf1b42f7ed356/)
   [Finished :arrow_down:0.17% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/9f501bcda19f44a096c12bd9990a15c0...ebeb4a2ccbc9400ab9c4508a615bf39a/)
   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] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/gdv_function_stubs.cc
##########
@@ -794,6 +795,53 @@ const char* gdv_fn_initcap_utf8(int64_t context, const char* data, int32_t data_
   *out_len = out_idx;
   return out;
 }
+
+int64_t gdv_fn_cast_intervalday_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                     const char* data, int data_len, bool in1_validity,
+                                     bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int64_t gdv_fn_cast_intervalday_utf8_int32(int64_t context_ptr, int64_t holder_ptr,
+                                           const char* data, int data_len,
+                                           bool in1_validity, int32_t /*suppress_errors*/,
+                                           bool /*in3_validity*/, bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int32_t gdv_fn_cast_intervalyear_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                      const char* data, int data_len, bool in1_validity,
+                                      bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalYearsHolder* holder =
+      reinterpret_cast<gandiva::IntervalYearsHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);

Review comment:
       Isn't this expensive as std::string does an allocate and a deep copy and then a free? not sure if it easy to change the interface to take something other than a string, though 

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (total_days & 0x00000000FFFFFFFF);
+  out |= ((module_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+// The operator will cast a generic string defined by the user into an interval of days.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int64_t IntervalDaysHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                       bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalDayFromMillis(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    return GetIntervalDayFromWeeks(period_in_weeks, out_valid);
+  }
+
+  std::string days_in_period;
+  std::string hours_in_period;
+  std::string minutes_in_period;
+  std::string seconds_in_period;
+  std::string ignored_string;  // string to store unnecessary captured groups
+  if (std::regex_match(data, period_not_contains_time)) {
+    if (RE2::FullMatch(data, iso8601_period_without_time, &ignored_string,
+                       &ignored_string, &days_in_period)) {
+      return GetIntervalDayFromCompletePeriod(days_in_period, hours_in_period,
+                                              minutes_in_period, seconds_in_period,
+                                              out_valid);
+    }
+
+    return_error(ctx, data);
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, iso8601_complete_period, &ignored_string, &ignored_string,
+                     &days_in_period, &hours_in_period, &minutes_in_period,
+                     &seconds_in_period)) {
+    return GetIntervalDayFromCompletePeriod(
+        days_in_period, hours_in_period, minutes_in_period, seconds_in_period, out_valid);
+  }
+
+  return_error(ctx, data);
+  return 0;
+}
+
+Status IntervalDaysHolder::Make(const FunctionNode& node,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  const std::string function_name("castINTERVALDAY");
+  return IntervalHolder<IntervalDaysHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalDaysHolder::Make(int32_t suppress_errors,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  return IntervalHolder<IntervalDaysHolder>::Make(suppress_errors, holder);
+}
+
+Status IntervalYearsHolder::Make(const FunctionNode& node,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  const std::string function_name("castINTERVALYEAR");
+  return IntervalHolder<IntervalYearsHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalYearsHolder::Make(int32_t suppress_errors,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  return IntervalHolder<IntervalYearsHolder>::Make(suppress_errors, holder);
+}
+
+// The operator will cast a generic string defined by the user into an interval of months.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int32_t IntervalYearsHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                        bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalYearFromNumber(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    *out_valid = true;

Review comment:
       did not understand this branch.  If we specify period in weeks (say > 52 weeks) won't it correspond to several months? or is this illegal..if it is illegal. shouln't we be returning an error?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       same as above

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       can this number be defined in a common place as MILLIS_IN_A_DAY either as a static const or a #define?

##########
File path: cpp/src/gandiva/gdv_function_stubs.cc
##########
@@ -794,6 +795,53 @@ const char* gdv_fn_initcap_utf8(int64_t context, const char* data, int32_t data_
   *out_len = out_idx;
   return out;
 }
+
+int64_t gdv_fn_cast_intervalday_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                     const char* data, int data_len, bool in1_validity,
+                                     bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int64_t gdv_fn_cast_intervalday_utf8_int32(int64_t context_ptr, int64_t holder_ptr,
+                                           const char* data, int data_len,
+                                           bool in1_validity, int32_t /*suppress_errors*/,
+                                           bool /*in3_validity*/, bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);

Review comment:
       can data_len be zero or less? if so, what happens to std::string construction?

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);

Review comment:
       why can't we exactly assert for the expected days and millis, by splitting the response to two 32 bits?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour

Review comment:
       feel it is more readable if a pre-processor macro or a static const is used, e,g MILLIS_IN_AN_HOUR and so on..

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;

Review comment:
       this is not millis_in_a_week, isn't it total_millis_in_all_weeks?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;

Review comment:
       did you mean modulo_millis?. module_millis is a confusing name. maybe remainder_millis or something like that?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;

Review comment:
       Can't millis in a week be pre-computed in the pre-processor stage,  either as a static const, e.g kMillisInAWeek = kMillisInADay * 7 or as a macro?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));

Review comment:
       here a static cast on stod is used for qty_days, but an stol is used for qty_days in a previous method. any reason for this?

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);

Review comment:
       please assert for exact expected value. similar comment for all EXPECT_GT below..

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (total_days & 0x00000000FFFFFFFF);
+  out |= ((module_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+// The operator will cast a generic string defined by the user into an interval of days.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int64_t IntervalDaysHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                       bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalDayFromMillis(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    return GetIntervalDayFromWeeks(period_in_weeks, out_valid);
+  }
+
+  std::string days_in_period;
+  std::string hours_in_period;
+  std::string minutes_in_period;
+  std::string seconds_in_period;
+  std::string ignored_string;  // string to store unnecessary captured groups
+  if (std::regex_match(data, period_not_contains_time)) {
+    if (RE2::FullMatch(data, iso8601_period_without_time, &ignored_string,
+                       &ignored_string, &days_in_period)) {
+      return GetIntervalDayFromCompletePeriod(days_in_period, hours_in_period,
+                                              minutes_in_period, seconds_in_period,
+                                              out_valid);
+    }
+
+    return_error(ctx, data);
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, iso8601_complete_period, &ignored_string, &ignored_string,
+                     &days_in_period, &hours_in_period, &minutes_in_period,
+                     &seconds_in_period)) {
+    return GetIntervalDayFromCompletePeriod(
+        days_in_period, hours_in_period, minutes_in_period, seconds_in_period, out_valid);
+  }
+
+  return_error(ctx, data);
+  return 0;
+}
+
+Status IntervalDaysHolder::Make(const FunctionNode& node,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  const std::string function_name("castINTERVALDAY");
+  return IntervalHolder<IntervalDaysHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalDaysHolder::Make(int32_t suppress_errors,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  return IntervalHolder<IntervalDaysHolder>::Make(suppress_errors, holder);
+}
+
+Status IntervalYearsHolder::Make(const FunctionNode& node,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  const std::string function_name("castINTERVALYEAR");
+  return IntervalHolder<IntervalYearsHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalYearsHolder::Make(int32_t suppress_errors,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  return IntervalHolder<IntervalYearsHolder>::Make(suppress_errors, holder);
+}
+
+// The operator will cast a generic string defined by the user into an interval of months.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int32_t IntervalYearsHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                        bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalYearFromNumber(data, out_valid);

Review comment:
       the comment above says this is a period in millis that is converted to months..but GetIntervalYearFromNumber is reading it directly as number of months. either you have to fix the comments or the code, no?

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));

Review comment:
       any reason why stod is used here, whereas stol is used in the previous. Is it because weeks can be specified in partial weeks (e.g 2.5). If that is the case, will doing just a static_cast loose some information?

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);

Review comment:
       I think we should assert for exact expected value

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass years and days and months to cast
+  data = "P12Y2M15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass days and months to cast
+  data = "P5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass all possible fields cast
+  data = "P2Y5M13DT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields cast
+  data = "PT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields to cast without hours
+  data = "PT42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only weeks to cast
+  data = "P25W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestMatchErrorsForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass an empty string
+  bool out_valid;
+  std::string data(" ");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass only days before years
+  data = "P15D12Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass years and days and months in wrong order
+  data = "P12M15D2Y";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Forget the P in the first position
+  data = "5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Use m instead M in the period format
+  data = "P2Y5M13DT10H42m21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Does not pass the T when defining only time fields
+  data = "P10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  // Pass weeks with other variables
+  data = "P2Y25W2M3D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_EQ(response_interval_yrs, 0);
+  EXPECT_FALSE(out_valid);
+  EXPECT_TRUE(execution_context_.has_error());
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingWeekFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("P1W");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 7);
+
+  data = "P10W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, 70);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalDay) {
+  std::shared_ptr<IntervalDaysHolder> interval_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_holder;
+
+  bool out_valid;
+  std::string data("1742461111");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  int64_t qty_days_in_response = 20;
+  int64_t qty_millis_in_response = 14461111;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P1Y1M1DT1H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 1;
+  qty_millis_in_response = 3661000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT48H1M1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 2;
+  qty_millis_in_response = 61000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "PT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  data = "P10DT1S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 10;
+  qty_millis_in_response = 1000;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+
+  data = "P0DT0S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  qty_days_in_response = 0;
+  qty_millis_in_response = 0;
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+  EXPECT_EQ(response, (qty_millis_in_response << 32) | qty_days_in_response);
+
+  execution_context_.Reset();
+}
+
+TEST_F(TestIntervalHolder, TestUsingCompleteFormatterForCastIntervalYear) {
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_years = *interval_years_holder;
+
+  bool out_valid;
+  std::string data("65851111");

Review comment:
       this looks a bit strange and confusing. If we are expecting input in months (and not millis), the comments in the operator above should be corrected. Also, if we expect months, something smaller like 623 months etc might be better as this looks like a large value for months.

##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       millis in a day seems to be occuring more than once, so surely a pre-processor or a static const might be better

##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  int32_t response_interval_yrs =
+      cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only years and days to cast
+  data = "P12Y15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass years and days and months to cast
+  data = "P12Y2M15D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass days and months to cast
+  data = "P5M13D";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass all possible fields cast
+  data = "P2Y5M13DT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields cast
+  data = "PT10H42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only time fields to cast without hours
+  data = "PT42M21S";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  // Pass only weeks to cast
+  data = "P25W";
+  response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);
+  EXPECT_TRUE(out_valid);
+  EXPECT_FALSE(execution_context_.has_error());
+
+  response_interval_yrs = cast_interval_year(&execution_context_, data, true, &out_valid);
+  EXPECT_GE(response_interval_yrs, 0);

Review comment:
       can this really be zero, for me 25 weeks is around 5 months.? if this is an error or if one is not expected to do castIntervalYear on a week format, what is expected?




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

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

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



[GitHub] [arrow] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour

Review comment:
       Done. I add the variables as static const.




-- 
This is an automated message from the 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] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,473 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response =
+      cast_interval_day(&execution_context_, data.data(), 8, true, &out_valid);
+  int64_t qty_days_in_response = 0;
+  int64_t qty_millis_in_response = 73834992;

Review comment:
       maybe 1 test case with > a day in millis? i.e > 86400000




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+  int64_t week_to_qty_days = 7;
+
+  int64_t millis_in_a_week = qty_weeks * week_to_qty_days * days_to_standard_millis;
+
+  int64_t qty_days = millis_in_a_week / days_to_standard_millis;
+  int64_t qty_millis = millis_in_a_week % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromCompletePeriod(
+    const std::string& days_in_period, const std::string& hours_in_period,
+    const std::string& minutes_in_period, const std::string& seconds_in_period,
+    bool* out_valid) {
+  int64_t qty_days = 0;
+  int64_t qty_hours = 0;
+  int64_t qty_minutes = 0;
+  int64_t qty_seconds = 0;
+
+  if (!days_in_period.empty()) {
+    qty_days = static_cast<int64_t>(std::stod(days_in_period));
+  }
+
+  if (!hours_in_period.empty()) {
+    qty_hours = static_cast<int64_t>(std::stod(hours_in_period));
+  }
+
+  if (!minutes_in_period.empty()) {
+    qty_minutes = static_cast<int64_t>(std::stod(minutes_in_period));
+  }
+
+  if (!seconds_in_period.empty()) {
+    qty_seconds = static_cast<int64_t>(std::stod(seconds_in_period));
+  }
+
+  int64_t millis_in_the_period = qty_hours * 3600000 +  // millis in a hour
+                                 qty_minutes * 60000 +  // millis in a minute
+                                 qty_seconds * 1000;
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t total_days = qty_days + (millis_in_the_period / days_to_standard_millis);
+  int64_t module_millis = millis_in_the_period % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (total_days & 0x00000000FFFFFFFF);
+  out |= ((module_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+// The operator will cast a generic string defined by the user into an interval of days.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int64_t IntervalDaysHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                       bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalDayFromMillis(data, out_valid);
+  }
+
+  std::string period_in_weeks;
+  if (RE2::FullMatch(data, iso8601_period_with_weeks, &period_in_weeks)) {
+    return GetIntervalDayFromWeeks(period_in_weeks, out_valid);
+  }
+
+  std::string days_in_period;
+  std::string hours_in_period;
+  std::string minutes_in_period;
+  std::string seconds_in_period;
+  std::string ignored_string;  // string to store unnecessary captured groups
+  if (std::regex_match(data, period_not_contains_time)) {
+    if (RE2::FullMatch(data, iso8601_period_without_time, &ignored_string,
+                       &ignored_string, &days_in_period)) {
+      return GetIntervalDayFromCompletePeriod(days_in_period, hours_in_period,
+                                              minutes_in_period, seconds_in_period,
+                                              out_valid);
+    }
+
+    return_error(ctx, data);
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, iso8601_complete_period, &ignored_string, &ignored_string,
+                     &days_in_period, &hours_in_period, &minutes_in_period,
+                     &seconds_in_period)) {
+    return GetIntervalDayFromCompletePeriod(
+        days_in_period, hours_in_period, minutes_in_period, seconds_in_period, out_valid);
+  }
+
+  return_error(ctx, data);
+  return 0;
+}
+
+Status IntervalDaysHolder::Make(const FunctionNode& node,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  const std::string function_name("castINTERVALDAY");
+  return IntervalHolder<IntervalDaysHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalDaysHolder::Make(int32_t suppress_errors,
+                                std::shared_ptr<IntervalDaysHolder>* holder) {
+  return IntervalHolder<IntervalDaysHolder>::Make(suppress_errors, holder);
+}
+
+Status IntervalYearsHolder::Make(const FunctionNode& node,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  const std::string function_name("castINTERVALYEAR");
+  return IntervalHolder<IntervalYearsHolder>::Make(node, holder, function_name);
+}
+
+Status IntervalYearsHolder::Make(int32_t suppress_errors,
+                                 std::shared_ptr<IntervalYearsHolder>* holder) {
+  return IntervalHolder<IntervalYearsHolder>::Make(suppress_errors, holder);
+}
+
+// The operator will cast a generic string defined by the user into an interval of months.
+// There are two formats of strings that are acceptable:
+//   - The period in millis: '238398430'
+//   - The period using a ISO8601 compatible format: 'P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W'
+int32_t IntervalYearsHolder::operator()(ExecutionContext* ctx, const std::string& data,
+                                        bool in_valid, bool* out_valid) {
+  *out_valid = false;
+
+  if (!in_valid) {
+    return 0;
+  }
+
+  if (RE2::FullMatch(data, period_only_contains_numbers)) {
+    return GetIntervalYearFromNumber(data, out_valid);

Review comment:
       The comment is incorrect, the number that the user defines is the quantity o months, I will fix it in the comment.




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;

Review comment:
       I put the value in a static const variable




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/gdv_function_stubs.cc
##########
@@ -794,6 +795,53 @@ const char* gdv_fn_initcap_utf8(int64_t context, const char* data, int32_t data_
   *out_len = out_idx;
   return out;
 }
+
+int64_t gdv_fn_cast_intervalday_utf8(int64_t context_ptr, int64_t holder_ptr,
+                                     const char* data, int data_len, bool in1_validity,
+                                     bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);
+  return (*holder)(context, data_as_string, in1_validity, out_valid);
+}
+
+int64_t gdv_fn_cast_intervalday_utf8_int32(int64_t context_ptr, int64_t holder_ptr,
+                                           const char* data, int data_len,
+                                           bool in1_validity, int32_t /*suppress_errors*/,
+                                           bool /*in3_validity*/, bool* out_valid) {
+  gandiva::ExecutionContext* context =
+      reinterpret_cast<gandiva::ExecutionContext*>(context_ptr);
+  gandiva::IntervalDaysHolder* holder =
+      reinterpret_cast<gandiva::IntervalDaysHolder*>(holder_ptr);
+  std::string data_as_string(data, data_len);

Review comment:
       I add a check to return an error in that cases https://github.com/apache/arrow/blob/637c60eaa40cb6ce3a11813e5c0eee2096721f4c/cpp/src/gandiva/interval_holder.cc#L144




-- 
This is an automated message from the 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] anthonylouisbsb commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder_test.cc
##########
@@ -0,0 +1,423 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <vector>
+
+#include "gandiva/execution_context.h"
+
+namespace gandiva {
+
+class TestIntervalHolder : public ::testing::Test {
+ protected:
+  ExecutionContext execution_context_;
+};
+
+TEST_F(TestIntervalHolder, TestMatchAllPeriods) {
+  std::shared_ptr<IntervalDaysHolder> interval_days_holder;
+  std::shared_ptr<IntervalYearsHolder> interval_years_holder;
+
+  auto status = IntervalDaysHolder::Make(0, &interval_days_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  status = IntervalYearsHolder::Make(0, &interval_years_holder);
+  EXPECT_EQ(status.ok(), true) << status.message();
+
+  auto& cast_interval_day = *interval_days_holder;
+  auto& cast_interval_year = *interval_years_holder;
+
+  // Pass only numbers to cast
+  bool out_valid;
+  std::string data("73834992");
+  int64_t response = cast_interval_day(&execution_context_, data, true, &out_valid);
+  EXPECT_GT(response, 0);

Review comment:
       I changed the response to match an exact 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] rkavanap commented on a change in pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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



##########
File path: cpp/src/gandiva/interval_holder.cc
##########
@@ -0,0 +1,277 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "gandiva/interval_holder.h"
+
+#include "gandiva/node.h"
+#include "gandiva/regex_util.h"
+
+namespace gandiva {
+
+// pre-compiled pattern for matching period that only have numbers
+static const RE2 period_only_contains_numbers(R"(\d+)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contains weeks.
+static const RE2 iso8601_complete_period(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)"
+    R"(T([[:digit:]]+H)?([[:digit:]]+M)?([[:digit:]]+S|[[:digit:]]+\.[[:digit:]]+S)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const RE2 iso8601_period_without_time(
+    R"(P([[:digit:]]+Y)?([[:digit:]]+M)?([[:digit:]]+D)?)");
+
+// pre-compiled pattern for matching periods in 8601 formats that not contain time
+// (hours, minutes and seconds) information.
+static const std::regex period_not_contains_time(R"(^((?!T).)*$)");
+
+// pre-compiled pattern for matching periods in 8601 formats that contains weeks inside
+// them. The ISO8601 specification defines that if the string contains a week, it can not
+// have other time granularities information, like day, years and months.
+static const RE2 iso8601_period_with_weeks(R"(P([[:digit:]]+W){1})");
+
+int64_t IntervalDaysHolder::GetIntervalDayFromMillis(const std::string& number_as_string,
+                                                     bool* out_valid) {
+  int64_t period_in_millis = std::stol(number_as_string);
+
+  // It considers that a day has exactly 24 hours of duration
+  int64_t days_to_standard_millis = 86400000;
+
+  int64_t qty_days = period_in_millis / days_to_standard_millis;
+  int64_t qty_millis = period_in_millis % days_to_standard_millis;
+
+  // The response is a 64-bit integer where the lower half of the bytes represents the
+  // number of the days and the other half represents the number of milliseconds.
+  int64_t out = (qty_days & 0x00000000FFFFFFFF);
+  out |= ((qty_millis << 32) & 0xFFFFFFFF00000000);
+
+  *out_valid = true;
+  return out;
+}
+
+int64_t IntervalDaysHolder::GetIntervalDayFromWeeks(const std::string& number_as_string,
+                                                    bool* out_valid) {
+  auto qty_weeks = static_cast<int64_t>(std::stod(number_as_string));

Review comment:
       Never mind. I think I understand what you are saying. While time intervals can be specified with decimals and we should accept it, due to the way Joda time works, decimal part should be ignored.
   It is still a bit confusing to me, but I think I get it..The only question then is, does the Joda time behaves this way where it accepts the fraction part of the decimal and then simply ignore it or does the Java version return an error if it sees a fraction 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] anthonylouisbsb commented on pull request #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   @rkavanap I applied the suggestions you made and I addressed some comments too. Can you take a look again?


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

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 #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   Benchmark runs are scheduled for baseline = 40ac81087a9d6d97dae4b11c28a147176d22dc54 and contender = adfb91302231418c5de53d427ddb7e8c3b6abee1. adfb91302231418c5de53d427ddb7e8c3b6abee1 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/c90fbedfe860480e921ccb7ef9a52257...ea88e1db3ed54ace950a544460b33c93/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/cce9eda123e64b89a3bc11f51eb8fbad...0aaddd68a6a5461489eaf1b42f7ed356/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/9f501bcda19f44a096c12bd9990a15c0...ebeb4a2ccbc9400ab9c4508a615bf39a/)
   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 #10137: ARROW-12516: [C++][Gandiva] Implements castINTERVALDAY(varchar) and castINTERVALYEAR(varchar) functions

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


   Benchmark runs are scheduled for baseline = 40ac81087a9d6d97dae4b11c28a147176d22dc54 and contender = adfb91302231418c5de53d427ddb7e8c3b6abee1. adfb91302231418c5de53d427ddb7e8c3b6abee1 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/c90fbedfe860480e921ccb7ef9a52257...ea88e1db3ed54ace950a544460b33c93/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/cce9eda123e64b89a3bc11f51eb8fbad...0aaddd68a6a5461489eaf1b42f7ed356/)
   [Finished :arrow_down:0.17% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/9f501bcda19f44a096c12bd9990a15c0...ebeb4a2ccbc9400ab9c4508a615bf39a/)
   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