Author: vikram
Date: Thu Apr  2 20:38:53 2015
New Revision: 1670961

URL: http://svn.apache.org/r1670961
Log:
HIVE-10201: Hive LLAP needs refactoring of the configuration class (Vikram 
Dixit K, reviewed by Siddharth Seth)

Modified:
    
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
    
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
    
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/registry/impl/LlapRegistryService.java
    
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
    
hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
    
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
    
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
    
hive/branches/llap/llap-server/src/test/org/apache/tez/dag/app/rm/TestLlapTaskSchedulerService.java

Modified: 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
 (original)
+++ 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
 Thu Apr  2 20:38:53 2015
@@ -32,7 +32,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.CompressionUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.cli.LlapOptionsProcessor.LlapOptions;
-import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.hadoop.hive.llap.io.api.impl.LlapInputFormat;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
@@ -121,7 +121,7 @@ public class LlapServiceDriver {
     if (options.getName() != null) {
       // update service registry configs - caveat: this has nothing to do with 
the actual settings as read by the AM
       // if needed, use --hiveconf llap.daemon.service.hosts=@llap0 to 
dynamically switch between instances
-      conf.set(LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS, "@" + 
options.getName());
+      conf.set(LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS, "@" + 
options.getName());
     }
 
     URL logger = conf.getResource("llap-daemon-log4j.properties");
@@ -188,17 +188,17 @@ public class LlapServiceDriver {
     configs.put(HiveConf.ConfVars.LLAP_ORC_CACHE_ALLOCATE_DIRECT.varname,
         HiveConf.getBoolVar(conf, 
HiveConf.ConfVars.LLAP_ORC_CACHE_ALLOCATE_DIRECT));
 
-    configs.put(LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB, 
conf.getInt(
-        LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB,
-        LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB_DEFAULT));
-
-    configs.put(LlapDaemonConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE, 
conf.getInt(
-        LlapDaemonConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE,
-        LlapDaemonConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE_DEFAULT));
-
-    configs.put(LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS, conf.getInt(
-        LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
-        LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS_DEFAULT));
+    configs.put(LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB, 
conf.getInt(
+        LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB,
+        LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB_DEFAULT));
+
+    configs.put(LlapConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE, conf.getInt(
+        LlapConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE,
+        LlapConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE_DEFAULT));
+
+    configs.put(LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS, conf.getInt(
+        LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
+        LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS_DEFAULT));
 
     configs.put(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
         conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1));

Modified: 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
 (original)
+++ 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
 Thu Apr  2 20:38:53 2015
@@ -24,8 +24,8 @@ import javax.management.ObjectName;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.hadoop.hive.llap.daemon.ContainerRunner;
-import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
 import org.apache.hadoop.hive.llap.daemon.registry.impl.LlapRegistryService;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
 import org.apache.hadoop.hive.llap.io.api.LlapIoProxy;
@@ -110,12 +110,12 @@ public class LlapDaemon extends Abstract
     this.shuffleHandlerConf.set(ShuffleHandler.SHUFFLE_HANDLER_LOCAL_DIRS,
         StringUtils.arrayToString(localDirs));
     
this.shuffleHandlerConf.setBoolean(ShuffleHandler.SHUFFLE_DIR_WATCHER_ENABLED, 
daemonConf
-        
.getBoolean(LlapDaemonConfiguration.LLAP_DAEMON_SHUFFLE_DIR_WATCHER_ENABLED,
-            
LlapDaemonConfiguration.LLAP_DAEMON_SHUFFLE_DIR_WATCHER_ENABLED_DEFAULT));
+        .getBoolean(LlapConfiguration.LLAP_DAEMON_SHUFFLE_DIR_WATCHER_ENABLED,
+            
LlapConfiguration.LLAP_DAEMON_SHUFFLE_DIR_WATCHER_ENABLED_DEFAULT));
 
     // Less frequently set parameter, not passing in as a param.
-    int numHandlers = 
daemonConf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS,
-        LlapDaemonConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS_DEFAULT);
+    int numHandlers = 
daemonConf.getInt(LlapConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS,
+        LlapConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS_DEFAULT);
     this.server = new LlapDaemonProtocolServerImpl(numHandlers, this, address, 
rpcPort);
 
     // Initialize the metric system
@@ -201,18 +201,18 @@ public class LlapDaemon extends Abstract
     try {
       // Cache settings will need to be setup in llap-daemon-site.xml - since 
the daemons don't read hive-site.xml
       // Ideally, these properties should be part of LlapDameonConf rather 
than HiveConf
-      LlapDaemonConfiguration daemonConf = new LlapDaemonConfiguration();
-       int numExecutors = 
daemonConf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
-           LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS_DEFAULT);
+      LlapConfiguration daemonConf = new LlapConfiguration();
+       int numExecutors = 
daemonConf.getInt(LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
+           LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS_DEFAULT);
        String[] localDirs =
-           
daemonConf.getTrimmedStrings(LlapDaemonConfiguration.LLAP_DAEMON_WORK_DIRS);
-       int rpcPort = 
daemonConf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT,
-           LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
+           
daemonConf.getTrimmedStrings(LlapConfiguration.LLAP_DAEMON_WORK_DIRS);
+       int rpcPort = daemonConf.getInt(LlapConfiguration.LLAP_DAEMON_RPC_PORT,
+           LlapConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
        int shufflePort = daemonConf
            .getInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 
ShuffleHandler.DEFAULT_SHUFFLE_PORT);
        long executorMemoryBytes = daemonConf
-           .getInt(LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB,
-               
LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB_DEFAULT) * 1024l * 
1024l;
+           .getInt(LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB,
+               LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB_DEFAULT) * 
1024l * 1024l;
        long cacheMemoryBytes =
            HiveConf.getLongVar(daemonConf, 
HiveConf.ConfVars.LLAP_ORC_CACHE_MAX_SIZE);
        boolean llapIoEnabled = HiveConf.getBoolVar(daemonConf, 
HiveConf.ConfVars.LLAP_IO_ENABLED);

Modified: 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/registry/impl/LlapRegistryService.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/registry/impl/LlapRegistryService.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/registry/impl/LlapRegistryService.java
 (original)
+++ 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/registry/impl/LlapRegistryService.java
 Thu Apr  2 20:38:53 2015
@@ -8,7 +8,7 @@ import java.net.UnknownHostException;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
@@ -56,7 +56,7 @@ public class LlapRegistryService extends
 
   @Override
   public void serviceInit(Configuration conf) {
-    String registryId = 
conf.getTrimmed(LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS);
+    String registryId = 
conf.getTrimmed(LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS);
     if (registryId.startsWith("@")) {
       LOG.info("Llap Registry is enabled with registryid: " + registryId);
       this.conf = new Configuration(conf);
@@ -86,16 +86,16 @@ public class LlapRegistryService extends
 
   public Endpoint getRpcEndpoint() {
     final int rpcPort =
-        conf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT,
-            LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
+        conf.getInt(LlapConfiguration.LLAP_DAEMON_RPC_PORT,
+            LlapConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
 
     return RegistryTypeUtils.ipcEndpoint("llap", new 
InetSocketAddress(hostname, rpcPort));
   }
 
   public Endpoint getShuffleEndpoint() {
     final int shufflePort =
-        conf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_YARN_SHUFFLE_PORT,
-            LlapDaemonConfiguration.LLAP_DAEMON_YARN_SHUFFLE_PORT_DEFAULT);
+        conf.getInt(LlapConfiguration.LLAP_DAEMON_YARN_SHUFFLE_PORT,
+            LlapConfiguration.LLAP_DAEMON_YARN_SHUFFLE_PORT_DEFAULT);
     // HTTP today, but might not be
     return RegistryTypeUtils.inetAddrEndpoint("shuffle", 
ProtocolTypes.PROTOCOL_TCP, hostname,
         shufflePort);
@@ -114,7 +114,7 @@ public class LlapRegistryService extends
       srv.addInternalEndpoint(getShuffleEndpoint());
 
       for (Map.Entry<String, String> kv : this.conf) {
-        if 
(kv.getKey().startsWith(LlapDaemonConfiguration.LLAP_DAEMON_PREFIX)) {
+        if (kv.getKey().startsWith(LlapConfiguration.LLAP_DAEMON_PREFIX)) {
           // TODO: read this somewhere useful, like the allocator
           srv.set(kv.getKey(), kv.getValue());
         }

Modified: 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
 (original)
+++ 
hive/branches/llap/llap-server/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
 Thu Apr  2 20:38:53 2015
@@ -26,7 +26,7 @@ import com.google.protobuf.ServiceExcept
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import 
org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
 import 
org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkResponseProto;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -72,8 +72,8 @@ public class LlapTaskCommunicator extend
   @Override
   public void serviceInit(Configuration conf) throws Exception {
     super.serviceInit(conf);
-    int numThreads = 
conf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_COMMUNICATOR_NUM_THREADS,
-        LlapDaemonConfiguration.LLAP_DAEMON_COMMUNICATOR_NUM_THREADS_DEFAULT);
+    int numThreads = 
conf.getInt(LlapConfiguration.LLAP_DAEMON_COMMUNICATOR_NUM_THREADS,
+        LlapConfiguration.LLAP_DAEMON_COMMUNICATOR_NUM_THREADS_DEFAULT);
     this.communicator = new TaskCommunicator(numThreads);
     this.communicator.init(conf);
   }

Modified: 
hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
 (original)
+++ 
hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/LlapTaskSchedulerService.java
 Thu Apr  2 20:38:53 2015
@@ -63,7 +63,7 @@ import org.apache.hadoop.yarn.api.record
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.hadoop.hive.llap.daemon.registry.impl.LlapRegistryService;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.tez.dag.api.TaskAttemptEndReason;
@@ -168,31 +168,31 @@ public class LlapTaskSchedulerService ex
     this.clock = appContext.getClock();
     this.containerFactory = new ContainerFactory(appContext, 
customAppIdIdentifier);
     this.memoryPerInstance = conf
-        .getInt(LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB,
-            
LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB_DEFAULT);
+        .getInt(LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB,
+            LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB_DEFAULT);
     this.coresPerInstance = conf
-        .getInt(LlapDaemonConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE,
-            LlapDaemonConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE_DEFAULT);
-    this.executorsPerInstance = 
conf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
-        LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS_DEFAULT);
+        .getInt(LlapConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE,
+            LlapConfiguration.LLAP_DAEMON_VCPUS_PER_INSTANCE_DEFAULT);
+    this.executorsPerInstance = 
conf.getInt(LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
+        LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS_DEFAULT);
     this.nodeReEnableTimeout = conf.getLong(
-        
LlapDaemonConfiguration.LLAP_DAEMON_TASK_SCHEDULER_NODE_REENABLE_TIMEOUT_MILLIS,
-        
LlapDaemonConfiguration.LLAP_DAEMON_TASK_SCHEDULER_NODE_REENABLE_TIMEOUT_MILLIS_DEFAULT);
+        
LlapConfiguration.LLAP_DAEMON_TASK_SCHEDULER_NODE_REENABLE_TIMEOUT_MILLIS,
+        
LlapConfiguration.LLAP_DAEMON_TASK_SCHEDULER_NODE_REENABLE_TIMEOUT_MILLIS_DEFAULT);
 
     int memoryPerExecutor = (int) (memoryPerInstance / (float) 
executorsPerInstance);
     int coresPerExecutor = (int) (coresPerInstance / (float) 
executorsPerInstance);
     this.resourcePerExecutor = Resource.newInstance(memoryPerExecutor, 
coresPerExecutor);
 
-    String instanceId = 
conf.getTrimmed(LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS);
+    String instanceId = 
conf.getTrimmed(LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS);
 
     Preconditions.checkNotNull(instanceId,
-        LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS + " must be 
defined");
+        LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS + " must be defined");
 
     if (!instanceId.startsWith("@")) { // Manual setup. Not via the service 
registry
       initFromRegistry = false;
-      String[] hosts = 
conf.getTrimmedStrings(LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS);
+      String[] hosts = 
conf.getTrimmedStrings(LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS);
       Preconditions.checkState(hosts != null && hosts.length != 0,
-          LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS + "must be 
defined");
+          LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS + "must be defined");
       for (String host : hosts) {
         NodeInfo nodeInfo = new NodeInfo(host, BACKOFF_FACTOR, clock);
         activeHosts.put(host, nodeInfo);
@@ -203,8 +203,8 @@ public class LlapTaskSchedulerService ex
       initFromRegistry = true;
     }
 
-    this.containerPort = 
conf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT,
-        LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
+    this.containerPort = conf.getInt(LlapConfiguration.LLAP_DAEMON_RPC_PORT,
+        LlapConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
     ExecutorService executorService = Executors.newFixedThreadPool(1,
         new 
ThreadFactoryBuilder().setDaemon(true).setNameFormat("LlapScheduler").build());
     executor = MoreExecutors.listeningDecorator(executorService);

Modified: 
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
 (original)
+++ 
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
 Thu Apr  2 20:38:53 2015
@@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.Service;
@@ -127,16 +128,16 @@ public class MiniLlapCluster extends Abs
   public void serviceStart() {
     llapDaemon.start();
 
-    
clusterSpecificConfiguration.set(LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS,
+    
clusterSpecificConfiguration.set(LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS,
         getServiceAddress().getHostName());
-    
clusterSpecificConfiguration.setInt(LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT,
+    clusterSpecificConfiguration.setInt(LlapConfiguration.LLAP_DAEMON_RPC_PORT,
         getServiceAddress().getPort());
 
     clusterSpecificConfiguration.setInt(
-        LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
+        LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS,
         numExecutorsPerService);
     clusterSpecificConfiguration.setLong(
-        LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB, 
execBytesPerService);
+        LlapConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB, 
execBytesPerService);
   }
 
   @Override

Modified: 
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
 (original)
+++ 
hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java
 Thu Apr  2 20:38:53 2015
@@ -21,8 +21,8 @@ import java.util.concurrent.atomic.Atomi
 
 import com.google.protobuf.ServiceException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.hadoop.hive.llap.daemon.ContainerRunner;
-import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
 import org.apache.hadoop.hive.llap.daemon.LlapDaemonProtocolBlockingPB;
 import 
org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SubmitWorkRequestProto;
 import org.junit.Test;
@@ -32,11 +32,11 @@ public class TestLlapDaemonProtocolServe
 
   @Test(timeout = 10000)
   public void test() throws ServiceException {
-    LlapDaemonConfiguration daemonConf = new LlapDaemonConfiguration();
-    int rpcPort = 
daemonConf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT,
-        LlapDaemonConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
-    int numHandlers = 
daemonConf.getInt(LlapDaemonConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS,
-        LlapDaemonConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS_DEFAULT);
+    LlapConfiguration daemonConf = new LlapConfiguration();
+    int rpcPort = daemonConf.getInt(LlapConfiguration.LLAP_DAEMON_RPC_PORT,
+        LlapConfiguration.LLAP_DAEMON_RPC_PORT_DEFAULT);
+    int numHandlers = 
daemonConf.getInt(LlapConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS,
+        LlapConfiguration.LLAP_DAEMON_RPC_NUM_HANDLERS_DEFAULT);
     LlapDaemonProtocolServerImpl server =
         new LlapDaemonProtocolServerImpl(numHandlers, 
mock(ContainerRunner.class),
             new AtomicReference<InetSocketAddress>(), rpcPort);

Modified: 
hive/branches/llap/llap-server/src/test/org/apache/tez/dag/app/rm/TestLlapTaskSchedulerService.java
URL: 
http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/test/org/apache/tez/dag/app/rm/TestLlapTaskSchedulerService.java?rev=1670961&r1=1670960&r2=1670961&view=diff
==============================================================================
--- 
hive/branches/llap/llap-server/src/test/org/apache/tez/dag/app/rm/TestLlapTaskSchedulerService.java
 (original)
+++ 
hive/branches/llap/llap-server/src/test/org/apache/tez/dag/app/rm/TestLlapTaskSchedulerService.java
 Thu Apr  2 20:38:53 2015
@@ -32,7 +32,7 @@ import static org.mockito.Mockito.verify
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
+import org.apache.hadoop.hive.llap.configuration.LlapConfiguration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -211,9 +211,9 @@ public class TestLlapTaskSchedulerServic
 
     TestTaskSchedulerServiceWrapper(long disableTimeoutMillis) {
       conf = new Configuration();
-      conf.setStrings(LlapDaemonConfiguration.LLAP_DAEMON_SERVICE_HOSTS, 
HOST1, HOST2, HOST3);
-      conf.setInt(LlapDaemonConfiguration.LLAP_DAEMON_NUM_EXECUTORS, 4);
-      
conf.setLong(LlapDaemonConfiguration.LLAP_DAEMON_TASK_SCHEDULER_NODE_REENABLE_TIMEOUT_MILLIS,
 disableTimeoutMillis);
+      conf.setStrings(LlapConfiguration.LLAP_DAEMON_SERVICE_HOSTS, HOST1, 
HOST2, HOST3);
+      conf.setInt(LlapConfiguration.LLAP_DAEMON_NUM_EXECUTORS, 4);
+      
conf.setLong(LlapConfiguration.LLAP_DAEMON_TASK_SCHEDULER_NODE_REENABLE_TIMEOUT_MILLIS,
 disableTimeoutMillis);
 
       doReturn(clock).when(mockAppContext).getClock();
       doReturn(appAttemptId).when(mockAppContext).getApplicationAttemptId();


Reply via email to