You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ra...@apache.org on 2019/08/09 12:54:35 UTC

[arrow] branch master updated: ARROW-6137: [C++][Gandiva] Change output format of castVARCHAR(timestamp) in Gandiva

This is an automated email from the ASF dual-hosted git repository.

ravindra pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new efeecfb  ARROW-6137: [C++][Gandiva] Change output format of castVARCHAR(timestamp) in Gandiva
efeecfb is described below

commit efeecfb21f2f6c46ef9267ffec3a79ba8fb792e3
Author: Prudhvi Porandla <pr...@icloud.com>
AuthorDate: Fri Aug 9 18:24:10 2019 +0530

    ARROW-6137: [C++][Gandiva] Change output format of castVARCHAR(timestamp) in Gandiva
    
    Format timestamp to yyyy-MM-dd hh:mm:ss.sss
    
    Closes #5014 from pprudhvi/timestamp-to-string and squashes the following commits:
    
    3cfad34ba <Prudhvi Porandla> do not hardcode ts length
    f1867e815 <Prudhvi Porandla> add java unittest
    93c6bf490 <Prudhvi Porandla> use iomanip to format timestamp
    35e74318a <Prudhvi Porandla> uset inttypes.h
    c3ea53885 <Prudhvi Porandla> cast to long long int
    1e389e2a6 <Prudhvi Porandla> use snprintf, return const char*
    73c39f41e <Prudhvi Porandla> error message if out_len is negative
    838947307 <Prudhvi Porandla> return empty string if out_len is zero
    637562282 <Prudhvi Porandla> refactor
    124b6dc34 <Prudhvi Porandla> correct castVarchar(timestamp) method
    
    Authored-by: Prudhvi Porandla <pr...@icloud.com>
    Signed-off-by: Pindikura Ravindra <ra...@dremio.com>
---
 cpp/src/gandiva/precompiled/time.cc                | 37 ++++++++--
 cpp/src/gandiva/precompiled/time_test.cc           | 19 +++++
 cpp/src/gandiva/precompiled/types.h                |  1 +
 .../arrow/gandiva/evaluator/ProjectorTest.java     | 80 ++++++++++++++++++++++
 4 files changed, 131 insertions(+), 6 deletions(-)

diff --git a/cpp/src/gandiva/precompiled/time.cc b/cpp/src/gandiva/precompiled/time.cc
index a1792d1..187afd8 100644
--- a/cpp/src/gandiva/precompiled/time.cc
+++ b/cpp/src/gandiva/precompiled/time.cc
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <iomanip>
+#include <sstream>
 #include "./epoch_time_point.h"
 
 extern "C" {
@@ -689,19 +691,42 @@ timestamp castTIMESTAMP_utf8(int64_t context, const char* input, int32 length) {
 
 timestamp castTIMESTAMP_date64(date64 date_in_millis) { return date_in_millis; }
 
-char* castVARCHAR_timestamp_int64(int64 context, timestamp in, int64 length,
-                                  int32* out_len) {
-  std::string timestamp_str = std::to_string(in);
-  *out_len = static_cast<int32>(length);
+const char* castVARCHAR_timestamp_int64(int64 context, timestamp in, int64 length,
+                                        int32* out_len) {
+  int64 year = extractYear_timestamp(in);
+  int64 month = extractMonth_timestamp(in);
+  int64 day = extractDay_timestamp(in);
+  int64 hour = extractHour_timestamp(in);
+  int64 minute = extractMinute_timestamp(in);
+  int64 second = extractSecond_timestamp(in);
+  int64 millis = in % MILLIS_IN_SEC;
+
+  // format to yyyy-MM-dd hh:mm:ss.sss
+  std::stringstream s;
+  s << std::setfill('0') << std::setw(4) << year << "-" << std::setw(2) << month << "-"
+    << std::setw(2) << day << " " << std::setw(2) << hour << ":" << std::setw(2) << minute
+    << ":" << std::setw(2) << second << "." << std::setw(3) << millis;
+  std::string timestamp_str = s.str();
   int32 timestamp_str_len = static_cast<int32>(timestamp_str.length());
-  if (length > timestamp_str_len) {
+
+  *out_len = static_cast<int32>(length);
+  if (*out_len > timestamp_str_len) {
     *out_len = timestamp_str_len;
   }
+
+  if (*out_len <= 0) {
+    if (*out_len < 0) {
+      gdv_fn_context_set_error_msg(context, "Length of output string cannot be negative");
+    }
+    *out_len = 0;
+    return "";
+  }
+
   char* ret = reinterpret_cast<char*>(gdv_fn_context_arena_malloc(context, *out_len));
   if (ret == nullptr) {
     gdv_fn_context_set_error_msg(context, "Could not allocate memory for output string");
     *out_len = 0;
-    return nullptr;
+    return "";
   }
   memcpy(ret, timestamp_str.data(), *out_len);
   return ret;
diff --git a/cpp/src/gandiva/precompiled/time_test.cc b/cpp/src/gandiva/precompiled/time_test.cc
index f30198e..f44795e 100644
--- a/cpp/src/gandiva/precompiled/time_test.cc
+++ b/cpp/src/gandiva/precompiled/time_test.cc
@@ -636,4 +636,23 @@ TEST(TestTime, TestMonthsBetween) {
   }
 }
 
+TEST(TestTime, castVarcharTimestamp) {
+  ExecutionContext context;
+  int64_t context_ptr = reinterpret_cast<int64_t>(&context);
+  int32 out_len;
+  timestamp ts = StringToTimestamp("2000-05-01 10:20:34");
+  const char* out = castVARCHAR_timestamp_int64(context_ptr, ts, 30L, &out_len);
+  EXPECT_EQ(std::string(out, out_len), "2000-05-01 10:20:34.000");
+
+  out = castVARCHAR_timestamp_int64(context_ptr, ts, 19L, &out_len);
+  EXPECT_EQ(std::string(out, out_len), "2000-05-01 10:20:34");
+
+  out = castVARCHAR_timestamp_int64(context_ptr, ts, 0L, &out_len);
+  EXPECT_EQ(std::string(out, out_len), "");
+
+  ts = StringToTimestamp("2-05-01 0:0:4");
+  out = castVARCHAR_timestamp_int64(context_ptr, ts, 24L, &out_len);
+  EXPECT_EQ(std::string(out, out_len), "0002-05-01 00:00:04.000");
+}
+
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/precompiled/types.h b/cpp/src/gandiva/precompiled/types.h
index 2935ecf..d67943f 100644
--- a/cpp/src/gandiva/precompiled/types.h
+++ b/cpp/src/gandiva/precompiled/types.h
@@ -165,6 +165,7 @@ date64 castDATE_utf8(int64_t execution_context, const char* input, int32 length)
 
 timestamp castTIMESTAMP_utf8(int64_t execution_context, const char* input, int32 length);
 timestamp castTIMESTAMP_date64(date64);
+const char* castVARCHAR_timestamp_int64(int64_t, timestamp, int64, int32*);
 
 int64 truncate_int64_int32(int64 in, int32 out_scale);
 
diff --git a/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/ProjectorTest.java b/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/ProjectorTest.java
index 52eeb16..22ec900 100644
--- a/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/ProjectorTest.java
+++ b/java/gandiva/src/test/java/org/apache/arrow/gandiva/evaluator/ProjectorTest.java
@@ -1388,4 +1388,84 @@ public class ProjectorTest extends BaseEvaluatorTest {
 
     assertTrue(caughtException);
   }
+
+  @Test
+  public void testCastTimestampToString() throws Exception {
+    ArrowType timeStamp = new ArrowType.Timestamp(TimeUnit.MILLISECOND, "TZ");
+
+    Field tsField = Field.nullable("timestamp", timeStamp);
+    Field lenField = Field.nullable("outLength", int64);
+
+    TreeNode tsNode = TreeBuilder.makeField(tsField);
+    TreeNode lenNode = TreeBuilder.makeField(lenField);
+
+    TreeNode tsToString = TreeBuilder.makeFunction("castVARCHAR", Lists.newArrayList(tsNode, lenNode),
+        new ArrowType.Utf8());
+
+    Field resultField = Field.nullable("result", new ArrowType.Utf8());
+    List<ExpressionTree> exprs =
+        Lists.newArrayList(
+            TreeBuilder.makeExpression(tsToString, resultField));
+
+    Schema schema = new Schema(Lists.newArrayList(tsField, lenField));
+    Projector eval = Projector.make(schema, exprs);
+
+    int numRows = 5;
+    byte[] validity = new byte[] {(byte) 255};
+    String[] values =
+        new String[] {
+            "0007-01-01T01:00:00Z",
+            "2007-03-05T03:40:00Z",
+            "2008-05-31T13:55:00Z",
+            "2000-06-30T23:20:00Z",
+            "2000-07-10T20:30:00Z",
+        };
+    long[] lenValues =
+        new long[] {
+            23L, 24L, 22L, 0L, 4L
+        };
+
+    String[] expValues =
+        new String[] {
+            "0007-01-01 01:00:00.000",
+            "2007-03-05 03:40:00.000",
+            "2008-05-31 13:55:00.00",
+            "",
+            "2000",
+        };
+
+    ArrowBuf bufValidity = buf(validity);
+    ArrowBuf millisData = stringToMillis(values);
+    ArrowBuf lenValidity = buf(validity);
+    ArrowBuf lenData = longBuf(lenValues);
+
+    ArrowFieldNode fieldNode = new ArrowFieldNode(numRows, 0);
+    ArrowRecordBatch batch =
+        new ArrowRecordBatch(
+            numRows,
+            Lists.newArrayList(fieldNode, fieldNode),
+            Lists.newArrayList(bufValidity, millisData, lenValidity, lenData));
+
+    List<ValueVector> output = new ArrayList<>();
+    for (int i = 0; i < exprs.size(); i++) {
+      VarCharVector charVector = new VarCharVector(EMPTY_SCHEMA_PATH, allocator);
+
+      charVector.allocateNew(numRows * 23, numRows);
+      output.add(charVector);
+    }
+    eval.evaluate(batch, output);
+    eval.close();
+
+    for (ValueVector valueVector : output) {
+      VarCharVector charVector = (VarCharVector) valueVector;
+
+      for (int j = 0; j < numRows; j++) {
+        assertFalse(charVector.isNull(j));
+        assertEquals(expValues[j], new String(charVector.get(j)));
+      }
+    }
+
+    releaseRecordBatch(batch);
+    releaseValueVectors(output);
+  }
 }