dxichen commented on a change in pull request #1588:
URL: https://github.com/apache/samza/pull/1588#discussion_r823046976



##########
File path: samza-core/src/main/java/org/apache/samza/config/JobConfig.java
##########
@@ -63,6 +63,8 @@
   public static final String JOB_CONTAINER_COUNT = "job.container.count";
   static final int DEFAULT_JOB_CONTAINER_COUNT = 1;
   public static final String JOB_CONTAINER_THREAD_POOL_SIZE = 
"job.container.thread.pool.size";
+  public static final String JOB_CONTAINER_TASK_EXECUTOR_TYPE = 
"job.container.task.executor.type";

Review comment:
       s/type/factory in the config names and variable names

##########
File path: 
samza-core/src/test/java/org/apache/samza/task/TestDefaultTaskExecutorFactory.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.samza.task;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.samza.config.JobConfig.JOB_CONTAINER_THREAD_POOL_SIZE;
+
+
+/**
+ * Tests {@link DefaultTaskExecutorFactory}.
+ */
+public class TestDefaultTaskExecutorFactory {
+
+  @Test
+  public void testGetTaskExecutor() {
+    DefaultTaskExecutorFactory factory = new DefaultTaskExecutorFactory();
+
+    Map<String, String> mapConfig = new HashMap<>();
+    int poolSize = 12;
+    mapConfig.put(JOB_CONTAINER_THREAD_POOL_SIZE, String.valueOf(poolSize));
+    Config config = new MapConfig(mapConfig);
+
+    ExecutorService executor = factory.getTaskExecutor(config);
+
+    Assert.assertEquals(poolSize, ((ThreadPoolExecutor) 
executor).getCorePoolSize());

Review comment:
       Ideally in an addition like this we would also add a mock 
`TaskExecutorFactory` impl to validate the override behavior with reflection

##########
File path: 
samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
##########
@@ -456,8 +456,13 @@ object SamzaContainer extends Logging {
     samzaContainerMetrics.containerThreadPoolSize.set(threadPoolSize)
 
     val taskThreadPool = if (threadPoolSize > 0) {
-      Executors.newFixedThreadPool(threadPoolSize,
-        new ThreadFactoryBuilder().setNameFormat("Samza Container 
Thread-%d").build())
+      val taskExecutorFactoryClassName = jobConfig.getTaskExecutorFactory
+      val taskExecutorFactory = 
ReflectionUtil.getObj(taskExecutorFactoryClassName, 
classOf[TaskExecutorFactory])
+      if (taskExecutorFactory != null) {

Review comment:
       This check should not be needed ReflectionUtil will throw if the class 
is not found/constructed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to