You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2015/07/03 01:25:18 UTC

spark git commit: Revert "[SPARK-8784] [SQL] Add Python API for hex and unhex"

Repository: spark
Updated Branches:
  refs/heads/master 488bad319 -> e589e71a2


Revert "[SPARK-8784] [SQL] Add Python API for hex and unhex"

This reverts commit fc7aebd94a3c09657fc4dbded0997ed068304e0a.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e589e71a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e589e71a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e589e71a

Branch: refs/heads/master
Commit: e589e71a2914588985eaea799b52e2f6b4f1e9ae
Parents: 488bad3
Author: Reynold Xin <rx...@databricks.com>
Authored: Thu Jul 2 16:25:10 2015 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Thu Jul 2 16:25:10 2015 -0700

----------------------------------------------------------------------
 python/pyspark/sql/functions.py                 |  28 ----
 .../catalyst/analysis/FunctionRegistry.scala    |   2 +-
 .../spark/sql/catalyst/expressions/math.scala   | 142 +++++++++----------
 .../expressions/MathFunctionsSuite.scala        |  18 +--
 .../scala/org/apache/spark/sql/functions.scala  |   2 +-
 5 files changed, 77 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/e589e71a/python/pyspark/sql/functions.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 8a470ce..12263e6 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -383,34 +383,6 @@ def randn(seed=None):
 
 @ignore_unicode_prefix
 @since(1.5)
-def hex(col):
-    """Computes hex value of the given column, which could be StringType,
-    BinaryType, IntegerType or LongType.
-
-    >>> sqlContext.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect()
-    [Row(hex(a)=u'414243', hex(b)=u'3')]
-    """
-    sc = SparkContext._active_spark_context
-    jc = sc._jvm.functions.hex(_to_java_column(col))
-    return Column(jc)
-
-
-@ignore_unicode_prefix
-@since(1.5)
-def unhex(col):
-    """Inverse of hex. Interprets each pair of characters as a hexadecimal number
-    and converts to the byte representation of number.
-
-    >>> sqlContext.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect()
-    [Row(unhex(a)=bytearray(b'ABC'))]
-    """
-    sc = SparkContext._active_spark_context
-    jc = sc._jvm.functions.unhex(_to_java_column(col))
-    return Column(jc)
-
-
-@ignore_unicode_prefix
-@since(1.5)
 def sha1(col):
     """Returns the hex string result of SHA-1.
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e589e71a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index ca87bcc..e7e4d1c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -160,7 +160,7 @@ object FunctionRegistry {
     expression[Substring]("substr"),
     expression[Substring]("substring"),
     expression[Upper]("ucase"),
-    expression[Unhex]("unhex"),
+    expression[UnHex]("unhex"),
     expression[Upper]("upper")
   )
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e589e71a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
index 1e09514..035980d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
@@ -227,20 +227,6 @@ case class Bin(child: Expression)
   }
 }
 
-object Hex {
-  val hexDigits = Array[Char](
-    '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'
-  ).map(_.toByte)
-
-  // lookup table to translate '0' -> 0 ... 'F'/'f' -> 15
-  val unhexDigits = {
-    val array = Array.fill[Byte](128)(-1)
-    (0 to 9).foreach(i => array('0' + i) = i.toByte)
-    (0 to 5).foreach(i => array('A' + i) = (i + 10).toByte)
-    (0 to 5).foreach(i => array('a' + i) = (i + 10).toByte)
-    array
-  }
-}
 
 /**
  * If the argument is an INT or binary, hex returns the number as a STRING in hexadecimal format.
@@ -272,18 +258,30 @@ case class Hex(child: Expression) extends UnaryExpression with Serializable  {
         case LongType => hex(num.asInstanceOf[Long])
         case IntegerType => hex(num.asInstanceOf[Integer].toLong)
         case BinaryType => hex(num.asInstanceOf[Array[Byte]])
-        case StringType => hex(num.asInstanceOf[UTF8String].getBytes)
+        case StringType => hex(num.asInstanceOf[UTF8String])
       }
     }
   }
 
-  private[this] def hex(bytes: Array[Byte]): UTF8String = {
-    val length = bytes.length
+  /**
+   * Converts every character in s to two hex digits.
+   */
+  private def hex(str: UTF8String): UTF8String = {
+    hex(str.getBytes)
+  }
+
+  private def hex(bytes: Array[Byte]): UTF8String = {
+    doHex(bytes, bytes.length)
+  }
+
+  private def doHex(bytes: Array[Byte], length: Int): UTF8String = {
     val value = new Array[Byte](length * 2)
     var i = 0
     while (i < length) {
-      value(i * 2) = Hex.hexDigits((bytes(i) & 0xF0) >> 4)
-      value(i * 2 + 1) = Hex.hexDigits(bytes(i) & 0x0F)
+      value(i * 2) = Character.toUpperCase(Character.forDigit(
+        (bytes(i) & 0xF0) >>> 4, 16)).toByte
+      value(i * 2 + 1) = Character.toUpperCase(Character.forDigit(
+        bytes(i) & 0x0F, 16)).toByte
       i += 1
     }
     UTF8String.fromBytes(value)
@@ -296,64 +294,14 @@ case class Hex(child: Expression) extends UnaryExpression with Serializable  {
     var len = 0
     do {
       len += 1
-      value(value.length - len) = Hex.hexDigits((numBuf & 0xF).toInt)
+      value(value.length - len) = Character.toUpperCase(Character
+        .forDigit((numBuf & 0xF).toInt, 16)).toByte
       numBuf >>>= 4
     } while (numBuf != 0)
     UTF8String.fromBytes(Arrays.copyOfRange(value, value.length - len, value.length))
   }
 }
 
-/**
- * Performs the inverse operation of HEX.
- * Resulting characters are returned as a byte array.
- */
-case class Unhex(child: Expression)
-  extends UnaryExpression with ExpectsInputTypes with Serializable {
-
-  override def nullable: Boolean = true
-  override def dataType: DataType = BinaryType
-  override def inputTypes: Seq[DataType] = Seq(BinaryType)
-
-  override def eval(input: InternalRow): Any = {
-    val num = child.eval(input)
-    if (num == null) {
-      null
-    } else {
-      unhex(num.asInstanceOf[UTF8String].getBytes)
-    }
-  }
-
-  private[this] def unhex(bytes: Array[Byte]): Array[Byte] = {
-    val out = new Array[Byte]((bytes.length + 1) >> 1)
-    var i = 0
-    if ((bytes.length & 0x01) != 0) {
-      // padding with '0'
-      if (bytes(0) < 0) {
-        return null
-      }
-      val v = Hex.unhexDigits(bytes(0))
-      if (v == -1) {
-        return null
-      }
-      out(0) = v
-      i += 1
-    }
-    // two characters form the hex value.
-    while (i < bytes.length) {
-      if (bytes(i) < 0 || bytes(i + 1) < 0) {
-        return null
-      }
-      val first = Hex.unhexDigits(bytes(i))
-      val second = Hex.unhexDigits(bytes(i + 1))
-      if (first == -1 || second == -1) {
-        return null
-      }
-      out(i / 2) = (((first << 4) | second) & 0xFF).toByte
-      i += 2
-    }
-    out
-  }
-}
 
 ////////////////////////////////////////////////////////////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////////////////////////////////
@@ -500,6 +448,58 @@ case class ShiftRight(left: Expression, right: Expression) extends BinaryExpress
   }
 }
 
+/**
+ * Performs the inverse operation of HEX.
+ * Resulting characters are returned as a byte array.
+ */
+case class UnHex(child: Expression) extends UnaryExpression with Serializable {
+
+  override def dataType: DataType = BinaryType
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    if (child.dataType.isInstanceOf[StringType] || child.dataType == NullType) {
+      TypeCheckResult.TypeCheckSuccess
+    } else {
+      TypeCheckResult.TypeCheckFailure(s"unHex accepts String type, not ${child.dataType}")
+    }
+  }
+
+  override def eval(input: InternalRow): Any = {
+    val num = child.eval(input)
+    if (num == null) {
+      null
+    } else {
+      unhex(num.asInstanceOf[UTF8String].getBytes)
+    }
+  }
+
+  private val unhexDigits = {
+    val array = Array.fill[Byte](128)(-1)
+    (0 to 9).foreach(i => array('0' + i) = i.toByte)
+    (0 to 5).foreach(i => array('A' + i) = (i + 10).toByte)
+    (0 to 5).foreach(i => array('a' + i) = (i + 10).toByte)
+    array
+  }
+
+  private def unhex(inputBytes: Array[Byte]): Array[Byte] = {
+    var bytes = inputBytes
+    if ((bytes.length & 0x01) != 0) {
+      bytes = '0'.toByte +: bytes
+    }
+    val out = new Array[Byte](bytes.length >> 1)
+    // two characters form the hex value.
+    var i = 0
+    while (i < bytes.length) {
+        val first = unhexDigits(bytes(i))
+        val second = unhexDigits(bytes(i + 1))
+        if (first == -1 || second == -1) { return null}
+        out(i / 2) = (((first << 4) | second) & 0xFF).toByte
+        i += 2
+    }
+    out
+  }
+}
+
 case class Hypot(left: Expression, right: Expression)
   extends BinaryMathExpression(math.hypot, "HYPOT")
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e589e71a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala
index 550c6e3..aa27fe3 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.{IntegerType, DataType, DoubleType, LongType}
 
 class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper {
 
@@ -252,15 +252,11 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper {
   }
 
   test("hex") {
-    checkEvaluation(Hex(Literal.create(null, IntegerType)), null)
     checkEvaluation(Hex(Literal(28)), "1C")
     checkEvaluation(Hex(Literal(-28)), "FFFFFFFFFFFFFFE4")
-    checkEvaluation(Hex(Literal.create(null, LongType)), null)
     checkEvaluation(Hex(Literal(100800200404L)), "177828FED4")
     checkEvaluation(Hex(Literal(-100800200404L)), "FFFFFFE887D7012C")
-    checkEvaluation(Hex(Literal.create(null, StringType)), null)
     checkEvaluation(Hex(Literal("helloHex")), "68656C6C6F486578")
-    checkEvaluation(Hex(Literal.create(null, BinaryType)), null)
     checkEvaluation(Hex(Literal("helloHex".getBytes())), "68656C6C6F486578")
     // scalastyle:off
     // Turn off scala style for non-ascii chars
@@ -269,15 +265,9 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper {
   }
 
   test("unhex") {
-    checkEvaluation(Unhex(Literal.create(null, StringType)), null)
-    checkEvaluation(Unhex(Literal("737472696E67")), "string".getBytes)
-    checkEvaluation(Unhex(Literal("")), new Array[Byte](0))
-    checkEvaluation(Unhex(Literal("F")), Array[Byte](15))
-    checkEvaluation(Unhex(Literal("ff")), Array[Byte](-1))
-    // scalastyle:off
-    // Turn off scala style for non-ascii chars
-    checkEvaluation(Unhex(Literal("E4B889E9878DE79A84")), "δΈ‰ι‡ηš„".getBytes("UTF-8"))
-    // scalastyle:on
+    checkEvaluation(UnHex(Literal("737472696E67")), "string".getBytes)
+    checkEvaluation(UnHex(Literal("")), new Array[Byte](0))
+    checkEvaluation(UnHex(Literal("0")), Array[Byte](0))
   }
 
   test("hypot") {

http://git-wip-us.apache.org/repos/asf/spark/blob/e589e71a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 4b1353f..4ee1fb8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -1061,7 +1061,7 @@ object functions {
    * @group math_funcs
    * @since 1.5.0
    */
-  def unhex(column: Column): Column = Unhex(column.expr)
+  def unhex(column: Column): Column = UnHex(column.expr)
 
   /**
    * Inverse of hex. Interprets each pair of characters as a hexadecimal number


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org