Github user chenghao-intel commented on a diff in the pull request:
https://github.com/apache/spark/pull/5497#discussion_r28283598
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala
---
@@ -0,0 +1,378 @@
+/*
+ * 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.execution
+
+import java.io._
+import java.nio.ByteBuffer
+import java.sql.Timestamp
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.serializer._
+import org.apache.spark.Logging
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow
+import org.apache.spark.sql.types._
+
+/**
+ * The serialization stream for SparkSqlSerializer2.
+ */
+private[sql] class Serializer2SerializationStream(
+ keySchema: Array[DataType],
+ valueSchema: Array[DataType],
+ out: OutputStream)
+ extends SerializationStream with Logging {
+
+ val rowOut = new DataOutputStream(out)
+ val writeKey =
SparkSqlSerializer2.createSerializationFunction(keySchema, rowOut)
+ val writeValue =
SparkSqlSerializer2.createSerializationFunction(valueSchema, rowOut)
+
+ def writeObject[T: ClassTag](t: T): SerializationStream = {
+ val kv = t.asInstanceOf[Product2[Row, Row]]
+ writeKey(kv._1)
+ writeValue(kv._2)
+
+ this
+ }
+
+ def flush(): Unit = {
+ rowOut.flush()
+ }
+
+ def close(): Unit = {
+ rowOut.close()
+ }
+}
+
+/**
+ * The deserialization stream for SparkSqlSerializer2.
+ */
+private[sql] class Serializer2DeserializationStream(
+ keySchema: Array[DataType],
+ valueSchema: Array[DataType],
+ in: InputStream)
+ extends DeserializationStream with Logging {
+
+ val rowIn = new DataInputStream(new BufferedInputStream(in))
+
+ val key = if (keySchema != null) new SpecificMutableRow(keySchema) else
null
+ val value = if (valueSchema != null) new SpecificMutableRow(valueSchema)
else null
+ val readKey =
SparkSqlSerializer2.createDeserializationFunction(keySchema, rowIn, key)
+ val readValue =
SparkSqlSerializer2.createDeserializationFunction(valueSchema, rowIn, value)
+
+ def readObject[T: ClassTag](): T = {
+ readKey()
+ readValue()
+
+ (key, value).asInstanceOf[T]
+ }
+
+ def close(): Unit = {
+ rowIn.close()
+ }
+}
+
+private[sql] class ShuffleSerializerInstance(
+ keySchema: Array[DataType],
+ valueSchema: Array[DataType])
+ extends SerializerInstance {
+
+ def serialize[T: ClassTag](t: T): ByteBuffer =
+ throw new UnsupportedOperationException("Not supported.")
+
+ def deserialize[T: ClassTag](bytes: ByteBuffer): T =
+ throw new UnsupportedOperationException("Not supported.")
+
+ def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T =
+ throw new UnsupportedOperationException("Not supported.")
+
+ def serializeStream(s: OutputStream): SerializationStream = {
+ new Serializer2SerializationStream(keySchema, valueSchema, s)
+ }
+
+ def deserializeStream(s: InputStream): DeserializationStream = {
+ new Serializer2DeserializationStream(keySchema, valueSchema, s)
+ }
+}
+
+/**
+ * SparkSqlSerializer2 is a special serializer that creates serialization
function and
+ * deserialization function based on the schema of data. It assumes that
values passed in
+ * are key/value pairs and values returned from it are also key/value
pairs.
+ * The schema of keys is represented by `keySchema` and that of values is
represented by
+ * `valueSchema`.
+ */
+private[sql] class SparkSqlSerializer2(keySchema: Array[DataType],
valueSchema: Array[DataType])
+ extends Serializer
+ with Logging
+ with Serializable{
+
+ def newInstance(): SerializerInstance = new
ShuffleSerializerInstance(keySchema, valueSchema)
--- End diff --
Instead of using `Array[DataType]`, how about using `Array[Expression]`?
Just for example, it's not necessary to serde a `Literal`.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]