You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/11/15 20:21:08 UTC

[36/50] [abbrv] hive git commit: HIVE-15167 : remove SerDe interface; undeprecate Deserializer and Serializer (Sergey Shelukhin, reviewed by Ashutosh Chauhan)

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index 2c1bb6f..4aac90a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -89,7 +89,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
-import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
@@ -1682,7 +1682,7 @@ public class TestInputOutputFormat {
           ObjectInspectorFactory.getReflectionObjectInspector(MyRow.class,
               ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
     }
-    SerDe serde = new OrcSerde();
+    AbstractSerDe serde = new OrcSerde();
     HiveOutputFormat<?, ?> outFormat = new OrcOutputFormat();
     org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter writer =
         outFormat.getHiveRecordWriter(conf, testFilePath, MyRow.class, true,
@@ -1816,7 +1816,7 @@ public class TestInputOutputFormat {
           ObjectInspectorFactory.getReflectionObjectInspector(NestedRow.class,
               ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
     }
-    SerDe serde = new OrcSerde();
+    AbstractSerDe serde = new OrcSerde();
     OutputFormat<?, ?> outFormat = new OrcOutputFormat();
     RecordWriter writer =
         outFormat.getRecordWriter(fs, conf, testFilePath.toString(),
@@ -1875,7 +1875,7 @@ public class TestInputOutputFormat {
         outFormat.getHiveRecordWriter(conf, testFilePath, MyRow.class, true,
             properties, Reporter.NULL);
     writer.close(true);
-    SerDe serde = new OrcSerde();
+    AbstractSerDe serde = new OrcSerde();
     SerDeUtils.initializeSerDe(serde, conf, properties, null);
     InputFormat<?,?> in = new OrcInputFormat();
     FileInputFormat.setInputPaths(conf, testFilePath.toString());
@@ -1941,7 +1941,7 @@ public class TestInputOutputFormat {
           ObjectInspectorFactory.getReflectionObjectInspector(StringRow.class,
               ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
     }
-    SerDe serde = new OrcSerde();
+    AbstractSerDe serde = new OrcSerde();
     HiveOutputFormat<?, ?> outFormat = new OrcOutputFormat();
     org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter writer =
         outFormat.getHiveRecordWriter(conf, testFilePath, StringRow.class,
@@ -2495,7 +2495,7 @@ public class TestInputOutputFormat {
           ObjectInspectorFactory.getReflectionObjectInspector(NestedRow.class,
               ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
     }
-    SerDe serde = new OrcSerde();
+    AbstractSerDe serde = new OrcSerde();
     OutputFormat<?, ?> outFormat = new OrcOutputFormat();
     conf.setInt("mapred.max.split.size", 50);
     RecordWriter writer =
@@ -2529,7 +2529,7 @@ public class TestInputOutputFormat {
   public void testSplitEliminationNullStats() throws Exception {
     Properties properties = new Properties();
     StructObjectInspector inspector = createSoi();
-    SerDe serde = new OrcSerde();
+    AbstractSerDe serde = new OrcSerde();
     OutputFormat<?, ?> outFormat = new OrcOutputFormat();
     conf.setInt("mapred.max.split.size", 50);
     RecordWriter writer =
@@ -3631,7 +3631,7 @@ public class TestInputOutputFormat {
     conf.setLong(HiveConf.ConfVars.HIVE_ORC_DEFAULT_STRIPE_SIZE.varname, newStripeSize);
     conf.setLong(HiveConf.ConfVars.MAPREDMAXSPLITSIZE.varname, newMaxSplitSize);
 
-    SerDe serde = new OrcSerde();
+    AbstractSerDe serde = new OrcSerde();
     HiveOutputFormat<?, ?> outFormat = new OrcOutputFormat();
     org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter writer =
       outFormat.getHiveRecordWriter(conf, testFilePath, MyRow.class, true,

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/java/org/apache/hadoop/hive/serde2/AbstractSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/AbstractSerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/AbstractSerDe.java
index 9434e91..049b35d 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/AbstractSerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/AbstractSerDe.java
@@ -32,7 +32,7 @@ import javax.annotation.Nullable;
  * new methods can be added in the underlying interface, SerDe, and only implementations
  * that need those methods overwrite it.
  */
-public abstract class AbstractSerDe implements SerDe {
+public abstract class AbstractSerDe implements Deserializer, Serializer {
 
   protected String configErrors;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java b/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java
index 3038037..a215092 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/DefaultFetchFormatter.java
@@ -35,24 +35,24 @@ import org.apache.hive.common.util.ReflectionUtil;
  */
 public class DefaultFetchFormatter<T> implements FetchFormatter<String> {
 
-  private SerDe mSerde;
+  private AbstractSerDe mSerde;
 
   @Override
   public void initialize(Configuration hconf, Properties props) throws SerDeException {
     mSerde = initializeSerde(hconf, props);
   }
 
-  private SerDe initializeSerde(Configuration conf, Properties props) throws SerDeException {
+  private AbstractSerDe initializeSerde(Configuration conf, Properties props) throws SerDeException {
     String serdeName = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE);
-    Class<? extends SerDe> serdeClass;
+    Class<? extends AbstractSerDe> serdeClass;
     try {
       serdeClass =
-          Class.forName(serdeName, true, JavaUtils.getClassLoader()).asSubclass(SerDe.class);
+          Class.forName(serdeName, true, JavaUtils.getClassLoader()).asSubclass(AbstractSerDe.class);
     } catch (ClassNotFoundException e) {
       throw new SerDeException(e);
     }
     // cast only needed for Hadoop 0.17 compatibility
-    SerDe serde = ReflectionUtil.newInstance(serdeClass, null);
+    AbstractSerDe serde = ReflectionUtil.newInstance(serdeClass, null);
     Properties serdeProps = new Properties();
     if (serde instanceof DelimitedJSONSerDe) {
       serdeProps.put(SERIALIZATION_FORMAT, props.getProperty(SERIALIZATION_FORMAT));

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java b/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
index df27db2..a1d3dd8 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
@@ -31,10 +31,9 @@ import org.apache.hadoop.io.Writable;
  * HiveDeserializer also provides the ObjectInspector which can be used to
  * inspect the internal structure of the object (that is returned by deserialize
  * function).
- * All deserializers should extend the abstract class AbstractDeserializer, and eventually
- * Deserializer interface should be removed
+ * All deserializers should extend the abstract class AbstractDeserializer.
+ * The interface is necessary for SerDes to be able to implement both Serializer and Deserializer.
  */
-@Deprecated
 public interface Deserializer {
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/java/org/apache/hadoop/hive/serde2/SerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/SerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/SerDe.java
deleted file mode 100644
index db15ce5..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/SerDe.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * 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.hadoop.hive.serde2;
-
-/**
- * A union of HiveDeserializer and HiveSerializer interface.
- *
- * If a developer wants his hive table to be read-only, then he just want to
- * return
- *
- * both readable and writable, then
- *
- * All serdes should extend the abstract class AbstractSerDe, and eventually SerDe interface
- * should be removed
- */
-@Deprecated
-public interface SerDe extends Deserializer, Serializer {
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java b/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java
index b39db89..3f07a86 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/Serializer.java
@@ -28,10 +28,9 @@ import org.apache.hadoop.io.Writable;
  * HiveSerializer is used to serialize data to a Hadoop Writable object. The
  * serialize In addition to the interface below, all implementations are assume
  * to have a ctor that takes a single 'Table' object as argument.
- * All serializers should extend the abstract class AbstractSerializer, and eventually
- * Serializer interface should be removed
+ * All serializers should extend the abstract class AbstractSerializer.
+ * The interface is necessary for SerDes to be able to implement both Serializer and Deserializer.
  */
-@Deprecated
 public interface Serializer {
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/java/org/apache/hadoop/hive/serde2/columnar/ColumnarSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/columnar/ColumnarSerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/columnar/ColumnarSerDe.java
index e32d9a6..36beaee 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/columnar/ColumnarSerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/columnar/ColumnarSerDe.java
@@ -88,7 +88,7 @@ public class ColumnarSerDe extends ColumnarSerDeBase {
   /**
    * Initialize the SerDe given the parameters.
    *
-   * @see SerDe#initialize(Configuration, Properties)
+   * @see AbstractSerDe#initialize(Configuration, Properties)
    */
   @Override
   public void initialize(Configuration conf, Properties tbl) throws SerDeException {
@@ -123,7 +123,7 @@ public class ColumnarSerDe extends ColumnarSerDeBase {
    * @param objInspector
    *          The ObjectInspector for the row object
    * @return The serialized Writable object
-   * @see SerDe#serialize(Object, ObjectInspector)
+   * @see AbstractSerDe#serialize(Object, ObjectInspector)
    */
   @Override
   public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException {

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java
index ac2d39f..17ecff1 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySimpleSerDe.java
@@ -105,7 +105,7 @@ public class LazySimpleSerDe extends AbstractEncodingAwareSerDe {
    * ","-separated column names columns.types: ",", ":", or ";"-separated column
    * types
    *
-   * @see SerDe#initialize(Configuration, Properties)
+   * @see AbstractSerDe#initialize(Configuration, Properties)
    */
   @Override
   public void initialize(Configuration job, Properties tbl)
@@ -141,7 +141,7 @@ public class LazySimpleSerDe extends AbstractEncodingAwareSerDe {
    * @param field
    *          the Writable that contains the data
    * @return The deserialized row Object.
-   * @see SerDe#deserialize(Writable)
+   * @see AbstractSerDe#deserialize(Writable)
    */
   @Override
   public Object doDeserialize(Writable field) throws SerDeException {
@@ -167,7 +167,7 @@ public class LazySimpleSerDe extends AbstractEncodingAwareSerDe {
   /**
    * Returns the Writable Class after serialization.
    *
-   * @see SerDe#getSerializedClass()
+   * @see AbstractSerDe#getSerializedClass()
    */
   @Override
   public Class<? extends Writable> getSerializedClass() {
@@ -186,7 +186,7 @@ public class LazySimpleSerDe extends AbstractEncodingAwareSerDe {
    *          The ObjectInspector for the row object
    * @return The serialized Writable object
    * @throws IOException
-   * @see SerDe#serialize(Object, ObjectInspector)
+   * @see AbstractSerDe#serialize(Object, ObjectInspector)
    */
   @Override
   public Writable doSerialize(Object obj, ObjectInspector objInspector)

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java b/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java
index 036be4e..646a29d 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/TestStatsSerde.java
@@ -137,7 +137,7 @@ public class TestStatsSerde extends TestCase {
     }
   }
 
-  private void deserializeAndSerializeLazyBinary(SerDe serDe, Object[] rows, ObjectInspector rowOI)
+  private void deserializeAndSerializeLazyBinary(AbstractSerDe serDe, Object[] rows, ObjectInspector rowOI)
       throws Throwable {
 
     BytesWritable bytes[] = new BytesWritable[rows.length];

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
index f1eeb2d..1c84fe6 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
@@ -24,7 +24,7 @@ import java.util.Random;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
-import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerdeRandomRowSource;
 import org.apache.hadoop.hive.serde2.VerifyFast;
 import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableDeserializeRead;
@@ -45,8 +45,8 @@ public class TestBinarySortableFast extends TestCase {
   private void testBinarySortableFast(
           SerdeRandomRowSource source, Object[][] rows,
           boolean[] columnSortOrderIsDesc, byte[] columnNullMarker, byte[] columnNotNullMarker,
-          SerDe serde, StructObjectInspector rowOI,
-          SerDe serde_fewer, StructObjectInspector writeRowOI,
+          AbstractSerDe serde, StructObjectInspector rowOI,
+          AbstractSerDe serde_fewer, StructObjectInspector writeRowOI,
           boolean ascending, PrimitiveTypeInfo[] primitiveTypeInfos,
           boolean useIncludeColumns, boolean doWriteFewerColumns, Random r) throws Throwable {
 
@@ -311,9 +311,9 @@ public class TestBinarySortableFast extends TestCase {
     order = StringUtils.leftPad("", columnCount, '+');
     String nullOrder;
     nullOrder = StringUtils.leftPad("", columnCount, 'a');
-    SerDe serde_ascending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order, nullOrder);
+    AbstractSerDe serde_ascending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order, nullOrder);
 
-    SerDe serde_ascending_fewer = null;
+    AbstractSerDe serde_ascending_fewer = null;
     if (doWriteFewerColumns) {
       String partialFieldNames = ObjectInspectorUtils.getFieldNames(writeRowStructObjectInspector);
       String partialFieldTypes = ObjectInspectorUtils.getFieldTypes(writeRowStructObjectInspector);
@@ -323,9 +323,9 @@ public class TestBinarySortableFast extends TestCase {
 
     order = StringUtils.leftPad("", columnCount, '-');
     nullOrder = StringUtils.leftPad("", columnCount, 'z');
-    SerDe serde_descending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order, nullOrder);
+    AbstractSerDe serde_descending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order, nullOrder);
 
-    SerDe serde_descending_fewer = null;
+    AbstractSerDe serde_descending_fewer = null;
     if (doWriteFewerColumns) {
       String partialFieldNames = ObjectInspectorUtils.getFieldNames(writeRowStructObjectInspector);
       String partialFieldTypes = ObjectInspectorUtils.getFieldTypes(writeRowStructObjectInspector);

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
index 935313b..6db2093 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
@@ -24,7 +24,7 @@ import java.util.Random;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.binarysortable.MyTestPrimitiveClass.ExtraTypeInfo;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -62,7 +62,7 @@ public class TestBinarySortableSerDe extends TestCase {
     return sb.toString();
   }
 
-  public static SerDe getSerDe(String fieldNames, String fieldTypes, String order, String nullOrder)
+  public static AbstractSerDe getSerDe(String fieldNames, String fieldTypes, String order, String nullOrder)
       throws Throwable {
     Properties schema = new Properties();
     schema.setProperty(serdeConstants.LIST_COLUMNS, fieldNames);
@@ -76,7 +76,7 @@ public class TestBinarySortableSerDe extends TestCase {
   }
 
   private void testBinarySortableSerDe(Object[] rows, ObjectInspector rowOI,
-      SerDe serde, boolean ascending) throws Throwable {
+      AbstractSerDe serde, boolean ascending) throws Throwable {
 
     ObjectInspector serdeOI = serde.getObjectInspector();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
index a1828c9..e62a80a 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinaryFast.java
@@ -23,7 +23,7 @@ import java.util.Random;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
-import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerdeRandomRowSource;
 import org.apache.hadoop.hive.serde2.VerifyFast;
 import org.apache.hadoop.hive.serde2.binarysortable.MyTestClass;
@@ -39,8 +39,8 @@ public class TestLazyBinaryFast extends TestCase {
 
   private void testLazyBinaryFast(
       SerdeRandomRowSource source, Object[][] rows,
-      SerDe serde, StructObjectInspector rowOI,
-      SerDe serde_fewer, StructObjectInspector writeRowOI,
+      AbstractSerDe serde, StructObjectInspector rowOI,
+      AbstractSerDe serde_fewer, StructObjectInspector writeRowOI,
       PrimitiveTypeInfo[] primitiveTypeInfos,
       boolean useIncludeColumns, boolean doWriteFewerColumns, Random r) throws Throwable {
 
@@ -242,9 +242,9 @@ public class TestLazyBinaryFast extends TestCase {
     String fieldNames = ObjectInspectorUtils.getFieldNames(rowStructObjectInspector);
     String fieldTypes = ObjectInspectorUtils.getFieldTypes(rowStructObjectInspector);
 
-    SerDe serde = TestLazyBinarySerDe.getSerDe(fieldNames, fieldTypes);
+    AbstractSerDe serde = TestLazyBinarySerDe.getSerDe(fieldNames, fieldTypes);
 
-    SerDe serde_fewer = null;
+    AbstractSerDe serde_fewer = null;
     if (doWriteFewerColumns) {
       String partialFieldNames = ObjectInspectorUtils.getFieldNames(writeRowStructObjectInspector);
       String partialFieldTypes = ObjectInspectorUtils.getFieldTypes(writeRowStructObjectInspector);

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java
index e54db95..0cd5736 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/lazybinary/TestLazyBinarySerDe.java
@@ -32,7 +32,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.binarysortable.MyTestClass;
 import org.apache.hadoop.hive.serde2.binarysortable.MyTestInnerStruct;
@@ -93,7 +93,7 @@ public class TestLazyBinarySerDe extends TestCase {
    * @return the initialized LazyBinarySerDe
    * @throws Throwable
    */
-  protected static SerDe getSerDe(String fieldNames, String fieldTypes) throws Throwable {
+  protected static AbstractSerDe getSerDe(String fieldNames, String fieldTypes) throws Throwable {
     Properties schema = new Properties();
     schema.setProperty(serdeConstants.LIST_COLUMNS, fieldNames);
     schema.setProperty(serdeConstants.LIST_COLUMN_TYPES, fieldTypes);
@@ -115,7 +115,7 @@ public class TestLazyBinarySerDe extends TestCase {
    * @throws Throwable
    */
   private void testLazyBinarySerDe(Object[] rows, ObjectInspector rowOI,
-      SerDe serde) throws Throwable {
+      AbstractSerDe serde) throws Throwable {
 
     ObjectInspector serdeOI = serde.getObjectInspector();
 
@@ -183,7 +183,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames1 = ObjectInspectorUtils.getFieldNames(rowOI1);
     String fieldTypes1 = ObjectInspectorUtils.getFieldTypes(rowOI1);
-    SerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
+    AbstractSerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
     serde1.getObjectInspector();
 
     StructObjectInspector rowOI2 = (StructObjectInspector) ObjectInspectorFactory
@@ -191,7 +191,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames2 = ObjectInspectorUtils.getFieldNames(rowOI2);
     String fieldTypes2 = ObjectInspectorUtils.getFieldTypes(rowOI2);
-    SerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
+    AbstractSerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
     ObjectInspector serdeOI2 = serde2.getObjectInspector();
 
     int num = 100;
@@ -226,7 +226,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames1 = ObjectInspectorUtils.getFieldNames(rowOI1);
     String fieldTypes1 = ObjectInspectorUtils.getFieldTypes(rowOI1);
-    SerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
+    AbstractSerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
     serde1.getObjectInspector();
 
     StructObjectInspector rowOI2 = (StructObjectInspector) ObjectInspectorFactory
@@ -234,7 +234,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames2 = ObjectInspectorUtils.getFieldNames(rowOI2);
     String fieldTypes2 = ObjectInspectorUtils.getFieldTypes(rowOI2);
-    SerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
+    AbstractSerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
     ObjectInspector serdeOI2 = serde2.getObjectInspector();
 
     int num = 100;
@@ -269,7 +269,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames1 = ObjectInspectorUtils.getFieldNames(rowOI1);
     String fieldTypes1 = ObjectInspectorUtils.getFieldTypes(rowOI1);
-    SerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
+    AbstractSerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
     serde1.getObjectInspector();
 
     StructObjectInspector rowOI2 = (StructObjectInspector) ObjectInspectorFactory
@@ -277,7 +277,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames2 = ObjectInspectorUtils.getFieldNames(rowOI2);
     String fieldTypes2 = ObjectInspectorUtils.getFieldTypes(rowOI2);
-    SerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
+    AbstractSerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
     ObjectInspector serdeOI2 = serde2.getObjectInspector();
 
     int num = 100;
@@ -313,7 +313,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames1 = ObjectInspectorUtils.getFieldNames(rowOI1);
     String fieldTypes1 = ObjectInspectorUtils.getFieldTypes(rowOI1);
-    SerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
+    AbstractSerDe serde1 = getSerDe(fieldNames1, fieldTypes1);
     serde1.getObjectInspector();
 
     StructObjectInspector rowOI2 = (StructObjectInspector) ObjectInspectorFactory
@@ -321,7 +321,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames2 = ObjectInspectorUtils.getFieldNames(rowOI2);
     String fieldTypes2 = ObjectInspectorUtils.getFieldTypes(rowOI2);
-    SerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
+    AbstractSerDe serde2 = getSerDe(fieldNames2, fieldTypes2);
     ObjectInspector serdeOI2 = serde2.getObjectInspector();
 
     int num = 100;
@@ -351,7 +351,7 @@ public class TestLazyBinarySerDe extends TestCase {
         ObjectInspectorOptions.JAVA);
     String fieldNames = ObjectInspectorUtils.getFieldNames(rowOI);
     String fieldTypes = ObjectInspectorUtils.getFieldTypes(rowOI);
-    SerDe serde = getSerDe(fieldNames, fieldTypes);
+    AbstractSerDe serde = getSerDe(fieldNames, fieldTypes);
     ObjectInspector serdeOI = serde.getObjectInspector();
 
     StructObjectInspector soi1 = (StructObjectInspector) serdeOI;

http://git-wip-us.apache.org/repos/asf/hive/blob/652ed7a7/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index abdf8cd..ba02c9c 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -60,7 +60,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.OperationLog;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -93,7 +93,7 @@ public class SQLOperation extends ExecuteStatementOperation {
   private CommandProcessorResponse response;
   private TableSchema resultSchema = null;
   private Schema mResultSchema = null;
-  private SerDe serde = null;
+  private AbstractSerDe serde = null;
   private boolean fetchStarted = false;
   private volatile MetricsScope currentSQLStateScope;
   // Display for WebUI.
@@ -575,7 +575,7 @@ public class SQLOperation extends ExecuteStatementOperation {
     return rowSet;
   }
 
-  private SerDe getSerDe() throws SQLException {
+  private AbstractSerDe getSerDe() throws SQLException {
     if (serde != null) {
       return serde;
     }