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 2015/01/29 21:40:35 UTC

svn commit: r1655848 [4/4] - in /hive/branches/llap/llap-server: ./ bin/ src/gen/ src/gen/protobuf/ src/gen/protobuf/gen-java/ src/gen/protobuf/gen-java/org/ src/gen/protobuf/gen-java/org/apache/ src/gen/protobuf/gen-java/org/apache/hadoop/ src/gen/pro...

Added: hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/DaemonTaskSchedulerService.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/DaemonTaskSchedulerService.java?rev=1655848&view=auto
==============================================================================
--- hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/DaemonTaskSchedulerService.java (added)
+++ hive/branches/llap/llap-server/src/java/org/apache/tez/dag/app/rm/DaemonTaskSchedulerService.java Thu Jan 29 20:40:34 2015
@@ -0,0 +1,326 @@
+/*
+ * Licensed 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.tez.dag.app.rm;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.primitives.Ints;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.hive.llap.daemon.LlapDaemonConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+
+
+// TODO Registration with RM - so that the AM is considered dead and restarted in the expiry interval - 10 minutes.
+
+public class DaemonTaskSchedulerService extends TaskSchedulerService {
+
+  private static final Log LOG = LogFactory.getLog(DaemonTaskSchedulerService.class);
+
+  private final ExecutorService appCallbackExecutor;
+  private final TaskSchedulerAppCallback appClientDelegate;
+  private final AppContext appContext;
+  private final List<String> serviceHosts;
+  private final ContainerFactory containerFactory;
+  private final Random random = new Random();
+
+  private final String clientHostname;
+  private final int clientPort;
+  private final String trackingUrl;
+  private final AtomicBoolean isStopped = new AtomicBoolean(false);
+  private final ConcurrentMap<Object, ContainerId> runningTasks =
+      new ConcurrentHashMap<Object, ContainerId>();
+
+  private final AMRMClientAsync<AMRMClient.ContainerRequest> amRmClient;
+
+  // Per daemon
+  private final int memoryPerInstance;
+  private final int coresPerInstance;
+  private final int executorsPerInstance;
+
+  // Per Executor Thread
+  private final Resource resourcePerExecutor;
+
+
+  public DaemonTaskSchedulerService(TaskSchedulerAppCallback appClient, AppContext appContext,
+                                    String clientHostname, int clientPort, String trackingUrl,
+                                    Configuration conf) {
+    // Accepting configuration here to allow setting up fields as final
+    super(DaemonTaskSchedulerService.class.getName());
+    this.appCallbackExecutor = createAppCallbackExecutorService();
+    this.appClientDelegate = createAppCallbackDelegate(appClient);
+    this.appContext = appContext;
+    this.serviceHosts = new LinkedList<String>();
+    this.containerFactory = new ContainerFactory(appContext);
+    this.memoryPerInstance = conf
+        .getInt(LlapDaemonConfiguration.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB,
+            LlapDaemonConfiguration.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);
+    this.clientHostname = clientHostname;
+    this.clientPort = clientPort;
+    this.trackingUrl = trackingUrl;
+
+    int memoryPerExecutor = (int) (memoryPerInstance / (float) executorsPerInstance);
+    int coresPerExecutor = (int) (coresPerInstance / (float) executorsPerInstance);
+    this.resourcePerExecutor = Resource.newInstance(memoryPerExecutor, coresPerExecutor);
+    this.amRmClient = TezAMRMClientAsync.createAMRMClientAsync(5000, new FakeAmRmCallbackHandler());
+
+    String[] hosts = conf.getTrimmedStrings(LlapDaemonConfiguration.LLAP_DAEMON_AM_SERVICE_HOSTS);
+    if (hosts == null || hosts.length == 0) {
+      hosts = new String[]{"localhost"};
+    }
+    for (String host : hosts) {
+      serviceHosts.add(host);
+    }
+
+    LOG.info("Running with configuration: " +
+        "memoryPerInstance=" + memoryPerInstance +
+        ", vcoresPerInstance=" + coresPerInstance +
+        ", executorsPerInstance=" + executorsPerInstance +
+        ", resourcePerInstanceInferred=" + resourcePerExecutor +
+        ", hosts=" + serviceHosts.toString());
+
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) {
+    amRmClient.init(conf);
+  }
+
+  @Override
+  public void serviceStart() {
+    amRmClient.start();
+    RegisterApplicationMasterResponse response;
+    try {
+      amRmClient.registerApplicationMaster(clientHostname, clientPort, trackingUrl);
+    } catch (YarnException e) {
+      throw new TezUncheckedException(e);
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
+  }
+
+  @Override
+  public void serviceStop() {
+    if (!this.isStopped.getAndSet(true)) {
+
+      try {
+        TaskSchedulerAppCallback.AppFinalStatus status = appClientDelegate.getFinalAppStatus();
+        amRmClient.unregisterApplicationMaster(status.exitStatus, status.exitMessage,
+            status.postCompletionTrackingUrl);
+      } catch (YarnException e) {
+        throw new TezUncheckedException(e);
+      } catch (IOException e) {
+        throw new TezUncheckedException(e);
+      }
+      appCallbackExecutor.shutdownNow();
+    }
+  }
+
+  @Override
+  public Resource getAvailableResources() {
+    // TODO This needs information about all running executors, and the amount of memory etc available across the cluster.
+    return Resource
+        .newInstance(Ints.checkedCast(serviceHosts.size() * memoryPerInstance),
+            serviceHosts.size() * coresPerInstance);
+  }
+
+  @Override
+  public int getClusterNodeCount() {
+    return serviceHosts.size();
+  }
+
+  @Override
+  public void resetMatchLocalityForAllHeldContainers() {
+  }
+
+  @Override
+  public Resource getTotalResources() {
+    return Resource
+        .newInstance(Ints.checkedCast(serviceHosts.size() * memoryPerInstance),
+            serviceHosts.size() * coresPerInstance);
+  }
+
+  @Override
+  public void blacklistNode(NodeId nodeId) {
+    LOG.info("DEBUG: BlacklistNode not supported");
+  }
+
+  @Override
+  public void unblacklistNode(NodeId nodeId) {
+    LOG.info("DEBUG: unBlacklistNode not supported");
+  }
+
+  @Override
+  public void allocateTask(Object task, Resource capability, String[] hosts, String[] racks,
+                           Priority priority, Object containerSignature, Object clientCookie) {
+    String host = selectHost(hosts);
+    Container container = containerFactory.createContainer(resourcePerExecutor, priority, host);
+    runningTasks.put(task, container.getId());
+    appClientDelegate.taskAllocated(task, clientCookie, container);
+  }
+
+
+  @Override
+  public void allocateTask(Object task, Resource capability, ContainerId containerId,
+                           Priority priority, Object containerSignature, Object clientCookie) {
+    String host = selectHost(null);
+    Container container = containerFactory.createContainer(resourcePerExecutor, priority, host);
+    runningTasks.put(task, container.getId());
+    appClientDelegate.taskAllocated(task, clientCookie, container);
+  }
+
+  @Override
+  public boolean deallocateTask(Object task, boolean taskSucceeded) {
+    ContainerId containerId = runningTasks.remove(task);
+    if (containerId == null) {
+      LOG.error("Could not determine ContainerId for task: " + task +
+          " . Could have hit a race condition. Ignoring." +
+          " The query may hang since this \"unknown\" container is now taking up a slot permanently");
+      return false;
+    }
+    appClientDelegate.containerBeingReleased(containerId);
+    return true;
+  }
+
+  @Override
+  public Object deallocateContainer(ContainerId containerId) {
+    LOG.info("DEBUG: Ignoring deallocateContainer for containerId: " + containerId);
+    return null;
+  }
+
+  @Override
+  public void setShouldUnregister() {
+
+  }
+
+  @Override
+  public boolean hasUnregistered() {
+    // Nothing to do. No registration involved.
+    return true;
+  }
+
+  private ExecutorService createAppCallbackExecutorService() {
+    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+        .setNameFormat("TaskSchedulerAppCaller #%d").setDaemon(true).build());
+  }
+
+  private TaskSchedulerAppCallback createAppCallbackDelegate(
+      TaskSchedulerAppCallback realAppClient) {
+    return new TaskSchedulerAppCallbackWrapper(realAppClient,
+        appCallbackExecutor);
+  }
+
+  private String selectHost(String[] requestedHosts) {
+    String host = null;
+    if (requestedHosts != null && requestedHosts.length > 0) {
+      Arrays.sort(requestedHosts);
+      host = requestedHosts[0];
+      LOG.info("Selected host: " + host + " from requested hosts: " + Arrays.toString(requestedHosts));
+    } else {
+      host = serviceHosts.get(random.nextInt(serviceHosts.size()));
+      LOG.info("Selected random host: " + host + " since the request contained no host information");
+    }
+    return host;
+  }
+
+  static class ContainerFactory {
+    final AppContext appContext;
+    AtomicInteger nextId;
+
+    public ContainerFactory(AppContext appContext) {
+      this.appContext = appContext;
+      this.nextId = new AtomicInteger(1);
+    }
+
+    public Container createContainer(Resource capability, Priority priority, String hostname) {
+      ApplicationAttemptId appAttemptId = appContext.getApplicationAttemptId();
+      ContainerId containerId = ContainerId.newInstance(appAttemptId, nextId.getAndIncrement());
+      NodeId nodeId = NodeId.newInstance(hostname, 0);
+      String nodeHttpAddress = "hostname:0";
+
+      Container container = Container.newInstance(containerId,
+          nodeId,
+          nodeHttpAddress,
+          capability,
+          priority,
+          null);
+
+      return container;
+    }
+  }
+
+  private static class FakeAmRmCallbackHandler implements AMRMClientAsync.CallbackHandler {
+
+    @Override
+    public void onContainersCompleted(List<ContainerStatus> statuses) {
+
+    }
+
+    @Override
+    public void onContainersAllocated(List<Container> containers) {
+
+    }
+
+    @Override
+    public void onShutdownRequest() {
+
+    }
+
+    @Override
+    public void onNodesUpdated(List<NodeReport> updatedNodes) {
+
+    }
+
+    @Override
+    public float getProgress() {
+      return 0;
+    }
+
+    @Override
+    public void onError(Throwable e) {
+
+    }
+  }
+}

Added: hive/branches/llap/llap-server/src/protobuf/LlapDaemonProtocol.proto
URL: http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/protobuf/LlapDaemonProtocol.proto?rev=1655848&view=auto
==============================================================================
--- hive/branches/llap/llap-server/src/protobuf/LlapDaemonProtocol.proto (added)
+++ hive/branches/llap/llap-server/src/protobuf/LlapDaemonProtocol.proto Thu Jan 29 20:40:34 2015
@@ -0,0 +1,40 @@
+/**
+ * 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.
+ */
+
+option java_package = "org.apache.hadoop.hive.llap.daemon.rpc";
+option java_outer_classname = "LlapDaemonProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+message RunContainerRequestProto {
+  optional string container_id_string = 1;
+  optional string am_host = 2;
+  optional int32 am_port = 3;
+  optional string token_identifier = 4;
+  optional bytes credentials_binary = 5;
+  optional string user = 6;
+  optional string application_id_string = 7;
+  optional int32 app_attempt_number = 8;
+}
+
+message RunContainerResponseProto {
+}
+
+service LlapDaemonProtocol {
+  rpc runContainer(RunContainerRequestProto) returns (RunContainerResponseProto);
+}

Added: 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=1655848&view=auto
==============================================================================
--- hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java (added)
+++ hive/branches/llap/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestLlapDaemonProtocolServerImpl.java Thu Jan 29 20:40:34 2015
@@ -0,0 +1,56 @@
+/*
+ * Licensed 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.llap.daemon.impl;
+
+import static org.mockito.Mockito.*;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.conf.Configuration;
+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.RunContainerRequestProto;
+import org.junit.Test;
+
+public class TestLlapDaemonProtocolServerImpl {
+
+
+  @Test(timeout = 10000)
+  public void test() throws ServiceException {
+    LlapDaemonConfiguration daemonConf = new LlapDaemonConfiguration();
+    LlapDaemonProtocolServerImpl server =
+        new LlapDaemonProtocolServerImpl(daemonConf, mock(ContainerRunner.class),
+            new AtomicReference<InetSocketAddress>());
+
+    try {
+      server.init(new Configuration());
+      server.start();
+      InetSocketAddress serverAddr = server.getBindAddress();
+
+      LlapDaemonProtocolBlockingPB client =
+          new LlapDaemonProtocolClientImpl(new Configuration(), serverAddr.getHostName(),
+              serverAddr.getPort());
+      client.runContainer(null,
+          RunContainerRequestProto.newBuilder().setAmHost("amhost")
+              .setAmPort(2000).build());
+
+    } finally {
+      server.stop();
+    }
+  }
+}

Added: hive/branches/llap/llap-server/src/test/resources/llap-daemon-log4j.properties
URL: http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/test/resources/llap-daemon-log4j.properties?rev=1655848&view=auto
==============================================================================
--- hive/branches/llap/llap-server/src/test/resources/llap-daemon-log4j.properties (added)
+++ hive/branches/llap/llap-server/src/test/resources/llap-daemon-log4j.properties Thu Jan 29 20:40:34 2015
@@ -0,0 +1,78 @@
+# 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.
+
+# Define some default values that can be overridden by system properties
+llap.daemon.root.logger=INFO,console
+llap.daemon.log.dir=.
+llap.daemon.log.file=llapdaemon.log
+
+# Define the root logger to the system property "llap.daemon.root.logger".
+log4j.rootLogger=${llap.daemon.root.logger}
+
+# Logging Threshold
+log4j.threshold=ALL
+
+# Null Appender
+log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
+
+#
+# Rolling File Appender - cap space usage at 5gb.
+#
+llap.daemon.log.maxfilesize=256MB
+llap.daemon.log.maxbackupindex=20
+log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+log4j.appender.RFA.File=${llap.daemon.log.dir}/${llap.daemon.log.file}
+
+log4j.appender.RFA.MaxFileSize=${llap.daemon.log.maxfilesize}
+log4j.appender.RFA.MaxBackupIndex=${llap.daemon.log.maxbackupindex}
+
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} [%t] %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${llap.daemon.log.dir}/${llap.daemon.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} [%t] %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this 
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} [%t] %p %c{2} : %m%n

Added: hive/branches/llap/llap-server/src/test/resources/llap-daemon-site.xml
URL: http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/test/resources/llap-daemon-site.xml?rev=1655848&view=auto
==============================================================================
--- hive/branches/llap/llap-server/src/test/resources/llap-daemon-site.xml (added)
+++ hive/branches/llap/llap-server/src/test/resources/llap-daemon-site.xml Thu Jan 29 20:40:34 2015
@@ -0,0 +1,27 @@
+<configuration>
+
+  <property>
+    <name>llap.daemon.work.dirs</name>
+    <value>/tmp/local/0</value>
+    <description>Set to yarn.nodemanager.local-dirs</description>
+  </property>
+
+  <property>
+    <name>llap.daemon.yarn.shuffle.port</name>
+    <value>15551</value>
+    <description>Set to the value on which the ShuffleHandler is running in YARN</description>
+  </property>
+
+  <property>
+    <name>llap.daemon.num.executors</name>
+    <value>4</value>
+    <description>Num executors for each daemon</description>
+  </property>
+
+  <property>
+    <name>llap.daemon.am.service.hosts</name>
+    <value>localhost</value>
+    <description>Comma separate list of nodes running daemons</description>
+  </property>
+
+</configuration>

Added: hive/branches/llap/llap-server/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/hive/branches/llap/llap-server/src/test/resources/log4j.properties?rev=1655848&view=auto
==============================================================================
--- hive/branches/llap/llap-server/src/test/resources/log4j.properties (added)
+++ hive/branches/llap/llap-server/src/test/resources/log4j.properties Thu Jan 29 20:40:34 2015
@@ -0,0 +1,19 @@
+#   Licensed 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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n