[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/20024


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159800597
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +665,17 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ArrayType(et, _) =>
+(c, evPrim, evNull) => {
+  val bufferTerm = ctx.freshName("bufferTerm")
--- End diff --

ok


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159799552
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +665,17 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ArrayType(et, _) =>
+(c, evPrim, evNull) => {
+  val bufferTerm = ctx.freshName("bufferTerm")
--- End diff --

super nit: In codegen we usually don't add a `term` postfix, just call it 
`buffer`, `array`, etc.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159774668
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -597,6 +619,44 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 """
   }
 
+  private[this] def codegenWriteArrayElemCode(et: DataType, ctx: 
CodegenContext): String = {
--- End diff --

ok, I'll update soon.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159682233
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -597,6 +619,44 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 """
   }
 
+  private[this] def codegenWriteArrayElemCode(et: DataType, ctx: 
CodegenContext): String = {
--- End diff --

`elementToString` needs a function because it's called twice.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159682007
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2775,4 +2773,53 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
   }
 }
   }
+
+  test("SPARK-22825 Cast array to string") {
--- End diff --

I think the unit test is good enough, we don't need this end-to-end test.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159681507
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -597,6 +619,44 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 """
   }
 
+  private[this] def codegenWriteArrayElemCode(et: DataType, ctx: 
CodegenContext): String = {
--- End diff --

oh wait, the returned function is only called once, I think we don't need 
to make it a function, but just return the code, e.g.
```
def writeArrayToStringBuilder(ctx: CodegenContext, et: DataType, arr: 
String, builder: String): String
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159680836
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -597,6 +619,44 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 """
   }
 
+  private[this] def codegenWriteArrayElemCode(et: DataType, ctx: 
CodegenContext): String = {
--- End diff --

It returns a function to write the array elements, maybe a better name is: 
`writeArrayToStringBuilderFunc`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159680098
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,80 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal byte buffer
--- End diff --

```
A helper class to write {@link UTF8String}s to an internal buffer and build 
the concatenated {@link UTF8String} at the end.
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159646388
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +671,17 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+(c, evPrim, evNull) => {
+  val bufferTerm = ctx.freshName("bufferTerm")
+  val bufferClass = classOf[UTF8StringBuilder].getName
+  val writeArrayToBuffer = codegenWriteArrayToBuffer(ar, ctx)
+  s"""
+ |$bufferClass $bufferTerm = new $bufferClass();
+ |$writeArrayToBuffer($c, $bufferTerm);
+ |$evPrim = $bufferTerm.toUTF8String();
+   """.stripMargin
--- End diff --

We can simplify this too
```
val elementToStringCode = castToStringCode(et, ctx)
val funcName = ctx.freshName("elementToString")
val elementToStringFunc = ctx.addNewFunction(funcName,
  s"""
 private UTF8String $funcName(${ctx.dataType(et)} element) {
   UTF8String elementStr = null;
   ${elementToStringCode("element", "elementStr", null /* resultIsNull 
won't be touched */)}
   return elementStr;
 }
   """)
...
$bufferClass $bufferTerm = new $bufferClass();
$bufferTerm.append("[");
if ($c.numElements > 0) {
  if (!$c.isNullAt(0)) {
$buffer.append($elementToStringFunc(${ctx.getValue(array, et, "0")}))
  }
  for (int $loopIndex = 1; $loopIndex < $arTerm.numElements(); 
$loopIndex++) ...
}
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159642734
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -206,6 +206,23 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
+  buildCast[ArrayData](_, array => {
+val res = new UTF8StringBuilder
--- End diff --

nit: `builder`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159642789
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -206,6 +206,23 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
--- End diff --

nit: `case ArrayType(et, _)`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159642702
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal byte buffer
+ * and get written data as `UTF8String`.
+ */
+public class UTF8StringBuilder {
+
+  private static final int ARRAY_MAX = 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH;
+
+  private byte[] buffer;
+  private int cursor = Platform.BYTE_ARRAY_OFFSET;
+
+  public UTF8StringBuilder() {
+// Since initial buffer size is 16 in `StringBuilder`, we set the same 
size here
+this.buffer = new byte[16];
+  }
+
+  // Grows the buffer by at least `neededSize`
+  private void grow(int neededSize) {
+if (neededSize > ARRAY_MAX - totalSize()) {
+  throw new UnsupportedOperationException(
+"Cannot grow internal buffer by size " + neededSize + " because 
the size after growing " +
+  "exceeds size limitation " + ARRAY_MAX);
+}
+final int length = totalSize() + neededSize;
+if (buffer.length < length) {
+  int newLength = length < ARRAY_MAX / 2 ? length * 2 : ARRAY_MAX;
+  final byte[] tmp = new byte[newLength];
+  Platform.copyMemory(
+buffer,
+Platform.BYTE_ARRAY_OFFSET,
+tmp,
+Platform.BYTE_ARRAY_OFFSET,
+totalSize());
+  buffer = tmp;
+}
+  }
+
+  public void append(UTF8String value) {
+grow(value.numBytes());
+value.writeToMemory(buffer, cursor);
+cursor += value.numBytes();
+  }
+
+  public void append(String value) {
+append(value.getBytes(StandardCharsets.UTF_8));
+  }
+
+  public void append(byte[] value) {
+grow(value.length);
+Platform.copyMemory(value, Platform.BYTE_ARRAY_OFFSET, buffer, cursor, 
value.length);
+cursor += value.length;
+  }
+
+  public UTF8String toUTF8String() {
--- End diff --

nit: `public UTF8String build()`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159642581
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal byte buffer
+ * and get written data as `UTF8String`.
+ */
+public class UTF8StringBuilder {
+
+  private static final int ARRAY_MAX = 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH;
+
+  private byte[] buffer;
+  private int cursor = Platform.BYTE_ARRAY_OFFSET;
+
+  public UTF8StringBuilder() {
+// Since initial buffer size is 16 in `StringBuilder`, we set the same 
size here
+this.buffer = new byte[16];
+  }
+
+  // Grows the buffer by at least `neededSize`
+  private void grow(int neededSize) {
+if (neededSize > ARRAY_MAX - totalSize()) {
+  throw new UnsupportedOperationException(
+"Cannot grow internal buffer by size " + neededSize + " because 
the size after growing " +
+  "exceeds size limitation " + ARRAY_MAX);
+}
+final int length = totalSize() + neededSize;
+if (buffer.length < length) {
+  int newLength = length < ARRAY_MAX / 2 ? length * 2 : ARRAY_MAX;
+  final byte[] tmp = new byte[newLength];
+  Platform.copyMemory(
+buffer,
+Platform.BYTE_ARRAY_OFFSET,
+tmp,
+Platform.BYTE_ARRAY_OFFSET,
+totalSize());
+  buffer = tmp;
+}
+  }
+
+  public void append(UTF8String value) {
+grow(value.numBytes());
+value.writeToMemory(buffer, cursor);
+cursor += value.numBytes();
+  }
+
+  public void append(String value) {
+append(value.getBytes(StandardCharsets.UTF_8));
+  }
+
+  public void append(byte[] value) {
+grow(value.length);
+Platform.copyMemory(value, Platform.BYTE_ARRAY_OFFSET, buffer, cursor, 
value.length);
+cursor += value.length;
+  }
+
+  public UTF8String toUTF8String() {
+final int len = totalSize();
+final byte[] bytes = new byte[len];
+Platform.copyMemory(buffer, Platform.BYTE_ARRAY_OFFSET, bytes, 
Platform.BYTE_ARRAY_OFFSET, len);
--- End diff --

why copy? we can do `UTF8String.fromBytes(buffer, 0, totalSize)`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159642451
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal byte buffer
+ * and get written data as `UTF8String`.
+ */
+public class UTF8StringBuilder {
+
+  private static final int ARRAY_MAX = 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH;
+
+  private byte[] buffer;
+  private int cursor = Platform.BYTE_ARRAY_OFFSET;
+
+  public UTF8StringBuilder() {
+// Since initial buffer size is 16 in `StringBuilder`, we set the same 
size here
+this.buffer = new byte[16];
+  }
+
+  // Grows the buffer by at least `neededSize`
+  private void grow(int neededSize) {
+if (neededSize > ARRAY_MAX - totalSize()) {
+  throw new UnsupportedOperationException(
+"Cannot grow internal buffer by size " + neededSize + " because 
the size after growing " +
+  "exceeds size limitation " + ARRAY_MAX);
+}
+final int length = totalSize() + neededSize;
+if (buffer.length < length) {
+  int newLength = length < ARRAY_MAX / 2 ? length * 2 : ARRAY_MAX;
+  final byte[] tmp = new byte[newLength];
+  Platform.copyMemory(
+buffer,
+Platform.BYTE_ARRAY_OFFSET,
+tmp,
+Platform.BYTE_ARRAY_OFFSET,
+totalSize());
+  buffer = tmp;
+}
+  }
+
+  public void append(UTF8String value) {
+grow(value.numBytes());
+value.writeToMemory(buffer, cursor);
+cursor += value.numBytes();
+  }
+
+  public void append(String value) {
+append(value.getBytes(StandardCharsets.UTF_8));
+  }
+
+  public void append(byte[] value) {
+grow(value.length);
+Platform.copyMemory(value, Platform.BYTE_ARRAY_OFFSET, buffer, cursor, 
value.length);
+cursor += value.length;
+  }
+
+  public UTF8String toUTF8String() {
+final int len = totalSize();
+final byte[] bytes = new byte[len];
+Platform.copyMemory(buffer, Platform.BYTE_ARRAY_OFFSET, bytes, 
Platform.BYTE_ARRAY_OFFSET, len);
+return UTF8String.fromBytes(bytes);
+  }
+
+  public int totalSize() {
--- End diff --

This doesn't need to be public


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159642422
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal byte buffer
+ * and get written data as `UTF8String`.
+ */
+public class UTF8StringBuilder {
+
+  private static final int ARRAY_MAX = 
ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH;
+
+  private byte[] buffer;
+  private int cursor = Platform.BYTE_ARRAY_OFFSET;
+
+  public UTF8StringBuilder() {
+// Since initial buffer size is 16 in `StringBuilder`, we set the same 
size here
+this.buffer = new byte[16];
+  }
+
+  // Grows the buffer by at least `neededSize`
+  private void grow(int neededSize) {
+if (neededSize > ARRAY_MAX - totalSize()) {
+  throw new UnsupportedOperationException(
+"Cannot grow internal buffer by size " + neededSize + " because 
the size after growing " +
+  "exceeds size limitation " + ARRAY_MAX);
+}
+final int length = totalSize() + neededSize;
+if (buffer.length < length) {
+  int newLength = length < ARRAY_MAX / 2 ? length * 2 : ARRAY_MAX;
+  final byte[] tmp = new byte[newLength];
+  Platform.copyMemory(
+buffer,
+Platform.BYTE_ARRAY_OFFSET,
+tmp,
+Platform.BYTE_ARRAY_OFFSET,
+totalSize());
+  buffer = tmp;
+}
+  }
+
+  public void append(UTF8String value) {
+grow(value.numBytes());
+value.writeToMemory(buffer, cursor);
+cursor += value.numBytes();
+  }
+
+  public void append(String value) {
+append(value.getBytes(StandardCharsets.UTF_8));
--- End diff --

This should be `append(UTF8String.fromString(value))`, then we can remove 
`append(byte[] value)`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159630335
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -199,13 +199,60 @@ case class Cast(child: Expression, dataType: 
DataType, timeZoneId: Option[String
 
   // [[func]] assumes the input is no longer null because eval already 
does the null check.
   @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = 
func(a.asInstanceOf[T])
+  @inline private[this] def buildWriter[T](
+  a: Any, buffer: UTF8StringBuilder, writer: (T, UTF8StringBuilder) => 
Unit): Unit = {
+writer(a.asInstanceOf[T], buffer)
+  }
+
+  private[this] def buildElemWriter(
+  from: DataType): (Any, UTF8StringBuilder) => Unit = from match {
+case BinaryType => buildWriter[Array[Byte]](_, _, (b, buf) => 
buf.append(b))
+case StringType => buildWriter[UTF8String](_, _, (b, buf) => 
buf.append(b))
+case DateType => buildWriter[Int](_, _,
+  (d, buf) => buf.append(DateTimeUtils.dateToString(d)))
+case TimestampType => buildWriter[Long](_, _,
+  (t, buf) => buf.append(DateTimeUtils.timestampToString(t)))
+case ar: ArrayType =>
+  buildWriter[ArrayData](_, _, (array, buf) => {
+ buf.append("[")
+if (array.numElements > 0) {
+  val writeElemToBuffer = buildElemWriter(ar.elementType)
+  writeElemToBuffer(array.get(0, ar.elementType), buf)
+  var i = 1
+  while (i < array.numElements) {
+buf.append(", ")
+writeElemToBuffer(array.get(i, ar.elementType), buf)
+i += 1
+  }
+}
+buf.append("]")
+  })
+case _ => buildWriter[Any](_, _, (o, buf) => 
buf.append(String.valueOf(o)))
+  }
 
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
 case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
+  buildCast[ArrayData](_, array => {
+val res = new UTF8StringBuilder
+res.append("[")
+if (array.numElements > 0) {
+  val writeElemToBuffer = buildElemWriter(ar.elementType)
+  writeElemToBuffer(array.get(0, ar.elementType), res)
--- End diff --

That's cuz a single string builder is reused for the whole conversion (In 
the suggested code, IIUC multiple string builders are created for nested 
arrays). But, yea. since simpler is better, I'll fix in that way.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159628763
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -199,13 +199,60 @@ case class Cast(child: Expression, dataType: 
DataType, timeZoneId: Option[String
 
   // [[func]] assumes the input is no longer null because eval already 
does the null check.
   @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = 
func(a.asInstanceOf[T])
+  @inline private[this] def buildWriter[T](
+  a: Any, buffer: UTF8StringBuilder, writer: (T, UTF8StringBuilder) => 
Unit): Unit = {
+writer(a.asInstanceOf[T], buffer)
+  }
+
+  private[this] def buildElemWriter(
+  from: DataType): (Any, UTF8StringBuilder) => Unit = from match {
+case BinaryType => buildWriter[Array[Byte]](_, _, (b, buf) => 
buf.append(b))
+case StringType => buildWriter[UTF8String](_, _, (b, buf) => 
buf.append(b))
+case DateType => buildWriter[Int](_, _,
+  (d, buf) => buf.append(DateTimeUtils.dateToString(d)))
+case TimestampType => buildWriter[Long](_, _,
+  (t, buf) => buf.append(DateTimeUtils.timestampToString(t)))
+case ar: ArrayType =>
+  buildWriter[ArrayData](_, _, (array, buf) => {
+ buf.append("[")
+if (array.numElements > 0) {
+  val writeElemToBuffer = buildElemWriter(ar.elementType)
+  writeElemToBuffer(array.get(0, ar.elementType), buf)
+  var i = 1
+  while (i < array.numElements) {
+buf.append(", ")
+writeElemToBuffer(array.get(i, ar.elementType), buf)
+i += 1
+  }
+}
+buf.append("]")
+  })
+case _ => buildWriter[Any](_, _, (o, buf) => 
buf.append(String.valueOf(o)))
+  }
 
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
 case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
+  buildCast[ArrayData](_, array => {
+val res = new UTF8StringBuilder
+res.append("[")
+if (array.numElements > 0) {
+  val writeElemToBuffer = buildElemWriter(ar.elementType)
+  writeElemToBuffer(array.get(0, ar.elementType), res)
--- End diff --

I don't get it, why do we need the `buildElemWriter`? I think it's as 
simple as
```
val toUTF8String = castToString(et)
builder.append(toUTF8String(array.get(0, et)).asInstanceOf[UTF8String])
var i = 1
while ...
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159627978
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal buffer
+ * and get a final concatenated string.
+ */
+public class UTF8StringBuilder {
+
+  private StringBuilder buffer;
--- End diff --

ok


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159627615
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal buffer
+ * and get a final concatenated string.
+ */
+public class UTF8StringBuilder {
+
+  private StringBuilder buffer;
--- End diff --

We should use the byte[] backed implementation. At the end we need a 
`UTF8String`, by wrapping `StringBuilder` we still have the unnecessary 
`String` -> `UTF8String` -> `String` conversion.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-04 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159606174
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java
 ---
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A helper class to write `UTF8String`, `String`, and `byte[]` data into 
an internal buffer
+ * and get a final concatenated string.
+ */
+public class UTF8StringBuilder {
+
+  private StringBuilder buffer;
--- End diff --

@cloud-fan In the latest commit, I just made this class by just wrapping 
`StringBuider`. Or, we can use byte[]-backed buffer implemenation like 
https://github.com/apache/spark/pull/20024/commits/91df078a99b6ec5f2063b2e73170336e3fe812d1#diff-ef9a1d6680a577c946ac779019a13beaR28.
Which one is preferred?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159590636
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -203,9 +203,26 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
 case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
+case StringType => buildCast[UTF8String](_, identity)
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
--- End diff --

yea, good idea. So, I'll add the builder class.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159590247
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +667,20 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+(c, evPrim, evNull) => {
+  val bufferTerm = ctx.freshName("bufferTerm")
+  val bufferClass = classOf[StringBuffer].getName
+  val writeArrayToBuffer = codegenWriteArrayToBuffer(ar, ctx)
+  s"""
+ |$bufferClass $bufferTerm = new $bufferClass();
+ |if (!$evNull) {
--- End diff --

ok.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159589454
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +667,20 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+(c, evPrim, evNull) => {
+  val bufferTerm = ctx.freshName("bufferTerm")
+  val bufferClass = classOf[StringBuffer].getName
--- End diff --

Since `StringBuffer` is used in  `stringExpressions` and 
`regexExpressions`, I just used this along with them. But, I'll update. (BTW, 
any reason to use `StringBuffer` in `stringExpressions` and `regexExpressions`?)


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159589202
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -203,9 +203,26 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
 case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
+case StringType => buildCast[UTF8String](_, identity)
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
--- End diff --

I think this can also simplify the codegen part, as we have some duplicated 
code to avoid this conversion.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159589126
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -203,9 +203,26 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
 case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
+case StringType => buildCast[UTF8String](_, identity)
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
--- End diff --

we may covert a string to `UTF8String` and then convert it back, which is 
inefficient. I think we should create a special `StringBuilder` for 
`UTF8String`, e.g.
```
class UTF8StringBuilder {
  public void append(UTF8String str)
}
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159588521
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +667,20 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+(c, evPrim, evNull) => {
+  val bufferTerm = ctx.freshName("bufferTerm")
+  val bufferClass = classOf[StringBuffer].getName
--- End diff --

why not use `StringBuilder`?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159588472
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +667,20 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+(c, evPrim, evNull) => {
+  val bufferTerm = ctx.freshName("bufferTerm")
+  val bufferClass = classOf[StringBuffer].getName
+  val writeArrayToBuffer = codegenWriteArrayToBuffer(ar, ctx)
+  s"""
+ |$bufferClass $bufferTerm = new $bufferClass();
+ |if (!$evNull) {
--- End diff --

looking at the document of `CastFunction`, the `evNull` is actually 
`resultIsNull`, it's not about input but about result, so we should not check 
it here.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159588015
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -203,9 +203,26 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
 case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
+case StringType => buildCast[UTF8String](_, identity)
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
+  buildCast[ArrayData](_, array => {
+val res = new StringBuilder
+res.append("[")
+if (array.numElements > 0) {
--- End diff --

yea, ok


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159587926
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -203,9 +203,26 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
   // UDFToString
   private[this] def castToString(from: DataType): Any => Any = from match {
 case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
+case StringType => buildCast[UTF8String](_, identity)
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
+  buildCast[ArrayData](_, array => {
+val res = new StringBuilder
+res.append("[")
+if (array.numElements > 0) {
--- End diff --

Actually I prefer your previous code style
```
val toStringFunc = castToString(ar.elementType)
if (array.numElements > 0) {
  res.append(toStringFunc(array.get(i, et)))
}
var i = 1
while (i < array.numElements) {
  res.append(", ")
  res.append(element...)
}
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159584638
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +723,22 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+val bufferClass = classOf[StringBuffer].getName
+val buffer = ctx.addMutableState(bufferClass, "buffer", v => s"$v 
= new $bufferClass();")
+val writeArrayToBuffer = codegenWriteArrayToBuffer(ar, buffer, ctx)
+val arrayToStringCode =
+  s"""
+ |if (!${ev.isNull}) {
--- End diff --

ya, I missed. Dropped `ev` now.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159584656
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -597,7 +605,114 @@ case class Cast(child: Expression, dataType: 
DataType, timeZoneId: Option[String
 """
   }
 
-  private[this] def castToStringCode(from: DataType, ctx: CodegenContext): 
CastFunction = {
+  private[this] def writeElemToBufferCode(
+  dataType: DataType,
+  buffer: String,
+  elemTerm: String,
+  ctx: CodegenContext): String = dataType match {
+case BinaryType => s"$buffer.append(new String($elemTerm))"
+case StringType => s"$buffer.append(new String($elemTerm.getBytes()))"
+case DateType => s"""$buffer.append(
+  
org.apache.spark.sql.catalyst.util.DateTimeUtils.dateToString($elemTerm))"""
+case TimestampType => s"""$buffer.append(
+  
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($elemTerm))"""
+case map: MapType => s"${codegenWriteMapToBuffer(map, buffer, 
ctx)}($elemTerm)"
+case ar: ArrayType => s"${codegenWriteArrayToBuffer(ar, buffer, 
ctx)}($elemTerm)"
+case st: StructType => s"${codegenWriteStructToBuffer(st, buffer, 
ctx)}($elemTerm)"
--- End diff --

ok, dropped.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159579422
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +723,22 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+val bufferClass = classOf[StringBuffer].getName
+val buffer = ctx.addMutableState(bufferClass, "buffer", v => s"$v 
= new $bufferClass();")
--- End diff --

ok


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159574368
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +723,22 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+val bufferClass = classOf[StringBuffer].getName
+val buffer = ctx.addMutableState(bufferClass, "buffer", v => s"$v 
= new $bufferClass();")
--- End diff --

We are casting array to string, which is expected to be slow. We don't need 
to save the object creation by introducing this mutable state. We can always 
create a new one.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159574300
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -608,6 +723,22 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 val tz = ctx.addReferenceObj("timeZone", timeZone)
 (c, evPrim, evNull) => s"""$evPrim = UTF8String.fromString(
   
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));"""
+  case ar: ArrayType =>
+val bufferClass = classOf[StringBuffer].getName
+val buffer = ctx.addMutableState(bufferClass, "buffer", v => s"$v 
= new $bufferClass();")
+val writeArrayToBuffer = codegenWriteArrayToBuffer(ar, buffer, ctx)
+val arrayToStringCode =
+  s"""
+ |if (!${ev.isNull}) {
--- End diff --

why is it? We are in `nullSafeCastFunction`, and ideally we don't need the 
`ev: ExprCode` parameter.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159574086
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -597,7 +605,114 @@ case class Cast(child: Expression, dataType: 
DataType, timeZoneId: Option[String
 """
   }
 
-  private[this] def castToStringCode(from: DataType, ctx: CodegenContext): 
CastFunction = {
+  private[this] def writeElemToBufferCode(
+  dataType: DataType,
+  buffer: String,
+  elemTerm: String,
+  ctx: CodegenContext): String = dataType match {
+case BinaryType => s"$buffer.append(new String($elemTerm))"
+case StringType => s"$buffer.append(new String($elemTerm.getBytes()))"
+case DateType => s"""$buffer.append(
+  
org.apache.spark.sql.catalyst.util.DateTimeUtils.dateToString($elemTerm))"""
+case TimestampType => s"""$buffer.append(
+  
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($elemTerm))"""
+case map: MapType => s"${codegenWriteMapToBuffer(map, buffer, 
ctx)}($elemTerm)"
+case ar: ArrayType => s"${codegenWriteArrayToBuffer(ar, buffer, 
ctx)}($elemTerm)"
+case st: StructType => s"${codegenWriteStructToBuffer(st, buffer, 
ctx)}($elemTerm)"
--- End diff --

since the interpreted version only support array type, we can also omit 
struct and map type here too, and add them in follow up PRs


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159572825
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -206,6 +208,11 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
+  buildCast[ArrayData](_, a => {
+val arrayData = CatalystTypeConverters.convertToScala(a, 
ar).asInstanceOf[WrappedArray[_]]
--- End diff --

ok, I'll update soon


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159572589
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -206,6 +208,11 @@ case class Cast(child: Expression, dataType: DataType, 
timeZoneId: Option[String
 case DateType => buildCast[Int](_, d => 
UTF8String.fromString(DateTimeUtils.dateToString(d)))
 case TimestampType => buildCast[Long](_,
   t => UTF8String.fromString(DateTimeUtils.timestampToString(t, 
timeZone)))
+case ar: ArrayType =>
+  buildCast[ArrayData](_, a => {
+val arrayData = CatalystTypeConverters.convertToScala(a, 
ar).asInstanceOf[WrappedArray[_]]
--- End diff --

let's avoid to do this, I'm thinking about
```
buildCast[ArrayData](_, array => {
  val res = new StringBuilder
  res.append("[")
  val toStringFunc = castToString(et)
  var i = 0
  while (i < array.numElements) {
res.append(toStringFunc(array.get(i, et)))
res.append(", ")
i += 1
  }
  // final cleanup and add "]"
})
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159473651
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/StringBuffer.java
 ---
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+
+/**
+ * A helper class to write array elements into a string buffer using 
`BufferHolder`.
+ */
+public class StringBuffer {
+
+  private BufferHolder buffer;
--- End diff --

Or, just use `java.lang.StringBuffer`?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2018-01-03 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r159451533
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/StringBuffer.java
 ---
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+package org.apache.spark.sql.catalyst.expressions.codegen;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+
+/**
+ * A helper class to write array elements into a string buffer using 
`BufferHolder`.
+ */
+public class StringBuffer {
+
+  private BufferHolder buffer;
--- End diff --

Currently, I reused `BufferHolder` for the string buffer though, probably 
we'd better to make another buffer implementation for this purpose?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2017-12-19 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r157911581
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2775,4 +2775,42 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
   }
 }
   }
+
+  test("SPARK-22825 Cast array to string") {
+// Check non-codegen path
+val df1 = sql("SELECT CAST(ARRAY(1, 2, 3, 4) AS STRING)")
+checkAnswer(df1, Row("[1, 2, 3, 4]"))
+val df2 = sql("SELECT CAST(ARRAY(ARRAY(1, 2), ARRAY(3, 4, 5), ARRAY(6, 
7)) AS STRING)")
+checkAnswer(df2, Row("[WrappedArray(1, 2), WrappedArray(3, 4, 5), 
WrappedArray(6, 7)]"))
--- End diff --

ok, thanks!


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2017-12-19 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/20024#discussion_r157900703
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2775,4 +2775,42 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
   }
 }
   }
+
+  test("SPARK-22825 Cast array to string") {
+// Check non-codegen path
+val df1 = sql("SELECT CAST(ARRAY(1, 2, 3, 4) AS STRING)")
+checkAnswer(df1, Row("[1, 2, 3, 4]"))
+val df2 = sql("SELECT CAST(ARRAY(ARRAY(1, 2), ARRAY(3, 4, 5), ARRAY(6, 
7)) AS STRING)")
+checkAnswer(df2, Row("[WrappedArray(1, 2), WrappedArray(3, 4, 5), 
WrappedArray(6, 7)]"))
--- End diff --

Hi, @maropu .
Could you put the result **after** this PR into PR description? So far, 
only **before** result is described.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #20024: [SPARK-22825][SQL] Fix incorrect results of Casti...

2017-12-19 Thread maropu
GitHub user maropu opened a pull request:

https://github.com/apache/spark/pull/20024

[SPARK-22825][SQL] Fix incorrect results of Casting Array to String

## What changes were proposed in this pull request?
This pr fixed the issue when casting arrays into strings;
```
scala> val df = 
spark.range(10).select('id.cast("integer")).agg(collect_list('id).as('ids))
scala> df.write.saveAsTable("t")
scala> sql("SELECT cast(ids as String) FROM t").show(false)
+--+
|ids   |
+--+
|org.apache.spark.sql.catalyst.expressions.UnsafeArrayData@8bc285df|
+--+
```

## How was this patch tested?
Added tests in `CastSuite` and `SQLQuerySuite`.



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/maropu/spark SPARK-22825

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/20024.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #20024


commit ed7f8857e54b0290cb136e9917f67a4809b3f138
Author: Takeshi Yamamuro 
Date:   2017-12-19T16:52:00Z

Fix a bug




---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org