You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2013/07/29 23:08:19 UTC

svn commit: r1508202 [48/48] - in /hive/branches/tez: ./ beeline/src/java/org/apache/hive/beeline/ cli/src/java/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hive/common/metrics/ common/src/java/org/apache/hadoop/hive/conf/ common/src/t...

Modified: hive/branches/tez/service/if/TCLIService.thrift
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/if/TCLIService.thrift?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/if/TCLIService.thrift (original)
+++ hive/branches/tez/service/if/TCLIService.thrift Mon Jul 29 21:08:03 2013
@@ -57,21 +57,25 @@ enum TTypeId {
   STRUCT_TYPE,
   UNION_TYPE,
   USER_DEFINED_TYPE,
-  DECIMAL_TYPE
+  DECIMAL_TYPE,
+  NULL_TYPE,
+  DATE_TYPE
 }
   
 const set<TTypeId> PRIMITIVE_TYPES = [
-  TTypeId.BOOLEAN_TYPE
-  TTypeId.TINYINT_TYPE
-  TTypeId.SMALLINT_TYPE
-  TTypeId.INT_TYPE
-  TTypeId.BIGINT_TYPE
-  TTypeId.FLOAT_TYPE
-  TTypeId.DOUBLE_TYPE
-  TTypeId.STRING_TYPE
-  TTypeId.TIMESTAMP_TYPE
+  TTypeId.BOOLEAN_TYPE,
+  TTypeId.TINYINT_TYPE,
+  TTypeId.SMALLINT_TYPE,
+  TTypeId.INT_TYPE,
+  TTypeId.BIGINT_TYPE,
+  TTypeId.FLOAT_TYPE,
+  TTypeId.DOUBLE_TYPE,
+  TTypeId.STRING_TYPE,
+  TTypeId.TIMESTAMP_TYPE,
   TTypeId.BINARY_TYPE,
-  TTypeId.DECIMAL_TYPE
+  TTypeId.DECIMAL_TYPE,
+  TTypeId.NULL_TYPE
+  TTypeId.DATE_TYPE
 ]
 
 const set<TTypeId> COMPLEX_TYPES = [
@@ -101,8 +105,10 @@ const map<TTypeId,string> TYPE_NAMES = {
   TTypeId.ARRAY_TYPE: "ARRAY",
   TTypeId.MAP_TYPE: "MAP",
   TTypeId.STRUCT_TYPE: "STRUCT",
-  TTypeId.UNION_TYPE: "UNIONTYPE"
-  TTypeId.DECIMAL_TYPE: "DECIMAL"
+  TTypeId.UNION_TYPE: "UNIONTYPE",
+  TTypeId.DECIMAL_TYPE: "DECIMAL",
+  TTypeId.NULL_TYPE: "NULL"
+  TTypeId.DATE_TYPE: "DATE"
 }
 
 // Thrift does not support recursively defined types or forward declarations,
@@ -285,7 +291,7 @@ union TColumnValue {
   4: TI32Value    i32Val       // INT
   5: TI64Value    i64Val       // BIGINT, TIMESTAMP
   6: TDoubleValue doubleVal    // FLOAT, DOUBLE
-  7: TStringValue stringVal    // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL
+  7: TStringValue stringVal    // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL, NULL
 }
 
 // Represents a row in a rowset.

Modified: hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_constants.cpp
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_constants.cpp?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_constants.cpp (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_constants.cpp Mon Jul 29 21:08:03 2013
@@ -22,6 +22,8 @@ TCLIServiceConstants::TCLIServiceConstan
   PRIMITIVE_TYPES.insert((TTypeId::type)8);
   PRIMITIVE_TYPES.insert((TTypeId::type)9);
   PRIMITIVE_TYPES.insert((TTypeId::type)15);
+  PRIMITIVE_TYPES.insert((TTypeId::type)16);
+  PRIMITIVE_TYPES.insert((TTypeId::type)17);
 
   COMPLEX_TYPES.insert((TTypeId::type)10);
   COMPLEX_TYPES.insert((TTypeId::type)11);
@@ -47,6 +49,8 @@ TCLIServiceConstants::TCLIServiceConstan
   TYPE_NAMES.insert(std::make_pair((TTypeId::type)12, "STRUCT"));
   TYPE_NAMES.insert(std::make_pair((TTypeId::type)13, "UNIONTYPE"));
   TYPE_NAMES.insert(std::make_pair((TTypeId::type)15, "DECIMAL"));
+  TYPE_NAMES.insert(std::make_pair((TTypeId::type)16, "NULL"));
+  TYPE_NAMES.insert(std::make_pair((TTypeId::type)17, "DATE"));
 
 }
 

Modified: hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp Mon Jul 29 21:08:03 2013
@@ -34,7 +34,9 @@ int _kTTypeIdValues[] = {
   TTypeId::STRUCT_TYPE,
   TTypeId::UNION_TYPE,
   TTypeId::USER_DEFINED_TYPE,
-  TTypeId::DECIMAL_TYPE
+  TTypeId::DECIMAL_TYPE,
+  TTypeId::NULL_TYPE,
+  TTypeId::DATE_TYPE
 };
 const char* _kTTypeIdNames[] = {
   "BOOLEAN_TYPE",
@@ -52,9 +54,11 @@ const char* _kTTypeIdNames[] = {
   "STRUCT_TYPE",
   "UNION_TYPE",
   "USER_DEFINED_TYPE",
-  "DECIMAL_TYPE"
+  "DECIMAL_TYPE",
+  "NULL_TYPE",
+  "DATE_TYPE"
 };
-const std::map<int, const char*> _TTypeId_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(16, _kTTypeIdValues, _kTTypeIdNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+const std::map<int, const char*> _TTypeId_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(18, _kTTypeIdValues, _kTTypeIdNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
 int _kTStatusCodeValues[] = {
   TStatusCode::SUCCESS_STATUS,

Modified: hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.h
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.h?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.h (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-cpp/TCLIService_types.h Mon Jul 29 21:08:03 2013
@@ -41,7 +41,9 @@ struct TTypeId {
     STRUCT_TYPE = 12,
     UNION_TYPE = 13,
     USER_DEFINED_TYPE = 14,
-    DECIMAL_TYPE = 15
+    DECIMAL_TYPE = 15,
+    NULL_TYPE = 16,
+    DATE_TYPE = 17
   };
 };
 

Modified: hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java Mon Jul 29 21:08:03 2013
@@ -46,6 +46,8 @@ public class TCLIServiceConstants {
     PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.TIMESTAMP_TYPE);
     PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BINARY_TYPE);
     PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE);
+    PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE);
+    PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE);
   }
 
   public static final Set<TTypeId> COMPLEX_TYPES = new HashSet<TTypeId>();
@@ -80,6 +82,8 @@ public class TCLIServiceConstants {
     TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.STRUCT_TYPE, "STRUCT");
     TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.UNION_TYPE, "UNIONTYPE");
     TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE, "DECIMAL");
+    TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE, "NULL");
+    TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE, "DATE");
   }
 
 }

Modified: hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeId.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeId.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeId.java (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeId.java Mon Jul 29 21:08:03 2013
@@ -27,7 +27,9 @@ public enum TTypeId implements org.apach
   STRUCT_TYPE(12),
   UNION_TYPE(13),
   USER_DEFINED_TYPE(14),
-  DECIMAL_TYPE(15);
+  DECIMAL_TYPE(15),
+  NULL_TYPE(16),
+  DATE_TYPE(17);
 
   private final int value;
 
@@ -80,6 +82,10 @@ public enum TTypeId implements org.apach
         return USER_DEFINED_TYPE;
       case 15:
         return DECIMAL_TYPE;
+      case 16:
+        return NULL_TYPE;
+      case 17:
+        return DATE_TYPE;
       default:
         return null;
     }

Modified: hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/constants.py
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/constants.py?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/constants.py (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/constants.py Mon Jul 29 21:08:03 2013
@@ -21,6 +21,8 @@ PRIMITIVE_TYPES = set([
     8,
     9,
     15,
+    16,
+    17,
 ])
 COMPLEX_TYPES = set([
     10,
@@ -49,4 +51,6 @@ TYPE_NAMES = {
     12 : "STRUCT",
     13 : "UNIONTYPE",
     15 : "DECIMAL",
+    16 : "NULL",
+    17 : "DATE",
 }

Modified: hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/ttypes.py
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/ttypes.py?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/ttypes.py (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-py/TCLIService/ttypes.py Mon Jul 29 21:08:03 2013
@@ -44,6 +44,8 @@ class TTypeId:
   UNION_TYPE = 13
   USER_DEFINED_TYPE = 14
   DECIMAL_TYPE = 15
+  NULL_TYPE = 16
+  DATE_TYPE = 17
 
   _VALUES_TO_NAMES = {
     0: "BOOLEAN_TYPE",
@@ -62,6 +64,8 @@ class TTypeId:
     13: "UNION_TYPE",
     14: "USER_DEFINED_TYPE",
     15: "DECIMAL_TYPE",
+    16: "NULL_TYPE",
+    17: "DATE_TYPE",
   }
 
   _NAMES_TO_VALUES = {
@@ -81,6 +85,8 @@ class TTypeId:
     "UNION_TYPE": 13,
     "USER_DEFINED_TYPE": 14,
     "DECIMAL_TYPE": 15,
+    "NULL_TYPE": 16,
+    "DATE_TYPE": 17,
   }
 
 class TStatusCode:

Modified: hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_constants.rb Mon Jul 29 21:08:03 2013
@@ -19,6 +19,8 @@ PRIMITIVE_TYPES = Set.new([
     8,
     9,
     15,
+    16,
+    17,
 ])
 
 COMPLEX_TYPES = Set.new([
@@ -50,5 +52,7 @@ TYPE_NAMES = {
     12 => %q"STRUCT",
     13 => %q"UNIONTYPE",
     15 => %q"DECIMAL",
+    16 => %q"NULL",
+    17 => %q"DATE",
 }
 

Modified: hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb (original)
+++ hive/branches/tez/service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb Mon Jul 29 21:08:03 2013
@@ -29,8 +29,10 @@ module TTypeId
   UNION_TYPE = 13
   USER_DEFINED_TYPE = 14
   DECIMAL_TYPE = 15
-  VALUE_MAP = {0 => "BOOLEAN_TYPE", 1 => "TINYINT_TYPE", 2 => "SMALLINT_TYPE", 3 => "INT_TYPE", 4 => "BIGINT_TYPE", 5 => "FLOAT_TYPE", 6 => "DOUBLE_TYPE", 7 => "STRING_TYPE", 8 => "TIMESTAMP_TYPE", 9 => "BINARY_TYPE", 10 => "ARRAY_TYPE", 11 => "MAP_TYPE", 12 => "STRUCT_TYPE", 13 => "UNION_TYPE", 14 => "USER_DEFINED_TYPE", 15 => "DECIMAL_TYPE"}
-  VALID_VALUES = Set.new([BOOLEAN_TYPE, TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, FLOAT_TYPE, DOUBLE_TYPE, STRING_TYPE, TIMESTAMP_TYPE, BINARY_TYPE, ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE, UNION_TYPE, USER_DEFINED_TYPE, DECIMAL_TYPE]).freeze
+  NULL_TYPE = 16
+  DATE_TYPE = 17
+  VALUE_MAP = {0 => "BOOLEAN_TYPE", 1 => "TINYINT_TYPE", 2 => "SMALLINT_TYPE", 3 => "INT_TYPE", 4 => "BIGINT_TYPE", 5 => "FLOAT_TYPE", 6 => "DOUBLE_TYPE", 7 => "STRING_TYPE", 8 => "TIMESTAMP_TYPE", 9 => "BINARY_TYPE", 10 => "ARRAY_TYPE", 11 => "MAP_TYPE", 12 => "STRUCT_TYPE", 13 => "UNION_TYPE", 14 => "USER_DEFINED_TYPE", 15 => "DECIMAL_TYPE", 16 => "NULL_TYPE", 17 => "DATE_TYPE"}
+  VALID_VALUES = Set.new([BOOLEAN_TYPE, TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, FLOAT_TYPE, DOUBLE_TYPE, STRING_TYPE, TIMESTAMP_TYPE, BINARY_TYPE, ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE, UNION_TYPE, USER_DEFINED_TYPE, DECIMAL_TYPE, NULL_TYPE, DATE_TYPE]).freeze
 end
 
 module TStatusCode

Modified: hive/branches/tez/service/src/java/org/apache/hive/service/cli/ColumnValue.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/java/org/apache/hive/service/cli/ColumnValue.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/java/org/apache/hive/service/cli/ColumnValue.java (original)
+++ hive/branches/tez/service/src/java/org/apache/hive/service/cli/ColumnValue.java Mon Jul 29 21:08:03 2013
@@ -18,6 +18,7 @@
 
 package org.apache.hive.service.cli;
 
+import java.sql.Date;
 import java.sql.Timestamp;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
@@ -36,6 +37,12 @@ import org.apache.hive.service.cli.thrif
  */
 public class ColumnValue {
 
+  public static final TColumnValue NULL = new TColumnValue();
+
+  static {
+    NULL.setStringVal(new TStringValue());
+  }
+
   // TODO: replace this with a non-Thrift implementation
   private final TColumnValue tColumnValue;
 
@@ -111,6 +118,14 @@ public class ColumnValue {
     return new ColumnValue(TColumnValue.stringVal(tStringValue));
   }
 
+  public static ColumnValue dateValue(Date value) {
+    TStringValue tStringValue = new TStringValue();
+    if (value != null) {
+      tStringValue.setValue(value.toString());
+    }
+    return new ColumnValue(TColumnValue.stringVal(tStringValue));
+  }
+
   public static ColumnValue timestampValue(Timestamp value) {
     TStringValue tStringValue = new TStringValue();
     if (value != null) {
@@ -145,6 +160,8 @@ public class ColumnValue {
       return doubleValue((Double)value);
     case STRING_TYPE:
       return stringValue((String)value);
+    case DATE_TYPE:
+      return dateValue((Date)value);
     case TIMESTAMP_TYPE:
       return timestampValue((Timestamp)value);
     case DECIMAL_TYPE:

Modified: hive/branches/tez/service/src/java/org/apache/hive/service/cli/Row.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/java/org/apache/hive/service/cli/Row.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/java/org/apache/hive/service/cli/Row.java (original)
+++ hive/branches/tez/service/src/java/org/apache/hive/service/cli/Row.java Mon Jul 29 21:08:03 2013
@@ -71,7 +71,11 @@ public class Row {
   public TRow toTRow() {
     TRow tRow = new TRow();
     for (ColumnValue columnValue : values) {
-      tRow.addToColVals(columnValue.toTColumnValue());
+      if (columnValue != null) {
+        tRow.addToColVals(columnValue.toTColumnValue());
+      } else {
+        tRow.addToColVals(ColumnValue.NULL);
+      }
     }
     return tRow;
   }

Modified: hive/branches/tez/service/src/java/org/apache/hive/service/cli/Type.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/service/src/java/org/apache/hive/service/cli/Type.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/service/src/java/org/apache/hive/service/cli/Type.java (original)
+++ hive/branches/tez/service/src/java/org/apache/hive/service/cli/Type.java Mon Jul 29 21:08:03 2013
@@ -27,6 +27,9 @@ import org.apache.hive.service.cli.thrif
  *
  */
 public enum Type {
+  NULL_TYPE("VOID",
+      java.sql.Types.NULL,
+      TTypeId.NULL_TYPE),
   BOOLEAN_TYPE("BOOLEAN",
       java.sql.Types.BOOLEAN,
       TTypeId.BOOLEAN_TYPE),
@@ -51,6 +54,9 @@ public enum Type {
   STRING_TYPE("STRING",
       java.sql.Types.VARCHAR,
       TTypeId.STRING_TYPE),
+  DATE_TYPE("DATE",
+      java.sql.Types.DATE,
+      TTypeId.DATE_TYPE),
   TIMESTAMP_TYPE("TIMESTAMP",
       java.sql.Types.TIMESTAMP,
       TTypeId.TIMESTAMP_TYPE),
@@ -77,7 +83,7 @@ public enum Type {
       java.sql.Types.VARCHAR,
       TTypeId.STRING_TYPE,
       true, false),
-  USER_DEFINED_TYPE(null,
+  USER_DEFINED_TYPE("USER_DEFINED",
       java.sql.Types.VARCHAR,
       TTypeId.STRING_TYPE,
       true, false);
@@ -123,6 +129,9 @@ public enum Type {
   }
 
   public static Type getType(String name) {
+    if (name == null) {
+      throw new IllegalArgumentException("Invalid type name: null");
+    }
     for (Type type : values()) {
       if (name.equalsIgnoreCase(type.name)) {
         return type;
@@ -207,6 +216,7 @@ public enum Type {
     switch (this) {
     case BOOLEAN_TYPE:
     case STRING_TYPE:
+    case DATE_TYPE:
     case TIMESTAMP_TYPE:
     case TINYINT_TYPE:
     case SMALLINT_TYPE:
@@ -241,6 +251,8 @@ public enum Type {
     case STRING_TYPE:
     case BINARY_TYPE:
       return Integer.MAX_VALUE;
+    case DATE_TYPE:
+      return 10;
     case TIMESTAMP_TYPE:
       return 30;
     default:

Modified: hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/DelegationTokenStore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/DelegationTokenStore.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/DelegationTokenStore.java (original)
+++ hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/DelegationTokenStore.java Mon Jul 29 21:08:03 2013
@@ -106,6 +106,8 @@ public interface DelegationTokenStore ex
    * and a potential scalability improvement would be to partition by master key id
    * @return
    */
-  List<DelegationTokenIdentifier> getAllDelegationTokenIdentifiers();
+  List<DelegationTokenIdentifier> getAllDelegationTokenIdentifiers() throws TokenStoreException;
+
+  void setStore(Object hmsHandler) throws TokenStoreException;
 
 }

Modified: hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java (original)
+++ hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge20S.java Mon Jul 29 21:08:03 2013
@@ -333,7 +333,7 @@ import static org.apache.hadoop.fs.Commo
      }
 
      @Override
-     public void startDelegationTokenSecretManager(Configuration conf)
+     public void startDelegationTokenSecretManager(Configuration conf, Object hms)
      throws IOException{
        long secretKeyInterval =
          conf.getLong(DELEGATION_KEY_UPDATE_INTERVAL_KEY,
@@ -345,10 +345,12 @@ import static org.apache.hadoop.fs.Commo
            conf.getLong(DELEGATION_TOKEN_RENEW_INTERVAL_KEY,
                         DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT);
 
+       DelegationTokenStore dts = getTokenStore(conf);
+       dts.setStore(hms);
        secretManager = new TokenStoreDelegationTokenSecretManager(secretKeyInterval,
              tokenMaxLifetime,
              tokenRenewInterval,
-             DELEGATION_TOKEN_GC_INTERVAL, getTokenStore(conf));
+             DELEGATION_TOKEN_GC_INTERVAL, dts);
        secretManager.startThreads();
      }
 

Modified: hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/MemoryTokenStore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/MemoryTokenStore.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/MemoryTokenStore.java (original)
+++ hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/MemoryTokenStore.java Mon Jul 29 21:08:03 2013
@@ -107,4 +107,9 @@ public class MemoryTokenStore implements
     //no-op
   }
 
+  @Override
+  public void setStore(Object hmsHandler) throws TokenStoreException {
+    // no-op
+  }
+
 }

Modified: hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/ZooKeeperTokenStore.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/ZooKeeperTokenStore.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/ZooKeeperTokenStore.java (original)
+++ hive/branches/tez/shims/src/common-secure/java/org/apache/hadoop/hive/thrift/ZooKeeperTokenStore.java Mon Jul 29 21:08:03 2013
@@ -74,7 +74,7 @@ public class ZooKeeperTokenStore impleme
         }
       }
     }
-    
+
   }
 
   /**
@@ -107,7 +107,7 @@ public class ZooKeeperTokenStore impleme
 
   /**
    * Create a ZooKeeper session that is in connected state.
-   * 
+   *
    * @param connectString ZooKeeper connect String
    * @param sessionTimeout ZooKeeper session timeout
    * @param connectTimeout milliseconds to wait for connection, 0 or negative value means no wait
@@ -147,7 +147,7 @@ public class ZooKeeperTokenStore impleme
     }
     return zk;
   }
-  
+
   /**
    * Create a path if it does not already exist ("mkdir -p")
    * @param zk ZooKeeper session
@@ -460,4 +460,9 @@ public class ZooKeeperTokenStore impleme
     }
   }
 
+  @Override
+  public void setStore(Object hmsHandler) throws TokenStoreException {
+    // no-op.
+  }
+
 }

Modified: hive/branches/tez/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java (original)
+++ hive/branches/tez/shims/src/common-secure/test/org/apache/hadoop/hive/thrift/TestHadoop20SAuthBridge.java Mon Jul 29 21:08:03 2013
@@ -35,7 +35,6 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
@@ -59,6 +58,12 @@ import org.apache.thrift.transport.TTran
 
 public class TestHadoop20SAuthBridge extends TestCase {
 
+  /**
+   * set to true when metastore token manager has intitialized token manager
+   * through call to HadoopThriftAuthBridge20S.Server.startDelegationTokenSecretManager
+   */
+  static volatile boolean isMetastoreTokenManagerInited;
+
   private static class MyHadoopThriftAuthBridge20S extends HadoopThriftAuthBridge20S {
     @Override
     public Server createServer(String keytabFile, String principalConf)
@@ -67,6 +72,8 @@ public class TestHadoop20SAuthBridge ext
       return new Server();
     }
 
+
+
     static class Server extends HadoopThriftAuthBridge20S.Server {
       public Server() throws TTransportException {
         super();
@@ -89,6 +96,14 @@ public class TestHadoop20SAuthBridge ext
       protected DelegationTokenStore getTokenStore(Configuration conf) throws IOException {
         return TOKEN_STORE;
       }
+
+      @Override
+      public void startDelegationTokenSecretManager(Configuration conf)
+      throws IOException{
+        super.startDelegationTokenSecretManager(conf);
+        isMetastoreTokenManagerInited = true;
+      }
+
     }
   }
 
@@ -120,6 +135,7 @@ public class TestHadoop20SAuthBridge ext
   }
 
   public void setup() throws Exception {
+    isMetastoreTokenManagerInited = false;
     int port = findFreePort();
     System.setProperty(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname,
         "true");
@@ -292,6 +308,9 @@ public class TestHadoop20SAuthBridge ext
     //metastore checks whether the authentication method is KERBEROS or not
     //for getDelegationToken, and the testcases don't use
     //kerberos, this needs to be done
+
+    waitForMetastoreTokenInit();
+
     HadoopThriftAuthBridge20S.Server.authenticationMethod
                              .set(AuthenticationMethod.KERBEROS);
     HadoopThriftAuthBridge20S.Server.remoteAddress.set(InetAddress.getLocalHost());
@@ -300,6 +319,20 @@ public class TestHadoop20SAuthBridge ext
             realUgi.getShortUserName());
   }
 
+  /**
+   * Wait for metastore to have initialized token manager
+   * This does not have to be done in other metastore test cases as they
+   * use metastore client which will retry few times on failure
+   * @throws InterruptedException
+   */
+  private void waitForMetastoreTokenInit() throws InterruptedException {
+    int waitAttempts = 30;
+    while(waitAttempts > 0 && !isMetastoreTokenManagerInited){
+      Thread.sleep(1000);
+      waitAttempts--;
+    }
+  }
+
   private void obtainTokenAndAddIntoUGI(UserGroupInformation clientUgi,
       String tokenSig) throws Exception {
     String tokenStrForm = getDelegationTokenStr(clientUgi, clientUgi);

Modified: hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java
URL: http://svn.apache.org/viewvc/hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java?rev=1508202&r1=1508201&r2=1508202&view=diff
==============================================================================
--- hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java (original)
+++ hive/branches/tez/shims/src/common/java/org/apache/hadoop/hive/thrift/HadoopThriftAuthBridge.java Mon Jul 29 21:08:03 2013
@@ -26,6 +26,7 @@ import org.apache.thrift.TProcessor;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.transport.TTransportFactory;
+
  /**
   * This class is only overridden by the secure hadoop shim. It allows
   * the Thrift SASL support to bridge to Hadoop's UserGroupInformation
@@ -74,8 +75,9 @@ import org.apache.thrift.transport.TTran
      public abstract TProcessor wrapProcessor(TProcessor processor);
      public abstract TProcessor wrapNonAssumingProcessor(TProcessor processor);
      public abstract InetAddress getRemoteAddress();
+     public abstract void startDelegationTokenSecretManager(Configuration conf,
+       Object hmsHandler) throws IOException;
      public abstract String getRemoteUser();
-     public abstract void startDelegationTokenSecretManager(Configuration conf) throws IOException;
      public abstract String getDelegationToken(String owner, String renewer) 
      throws IOException, InterruptedException;
      public abstract long renewDelegationToken(String tokenStrForm) throws IOException;