gyfora commented on a change in pull request #44:
URL:
https://github.com/apache/flink-kubernetes-operator/pull/44#discussion_r820508224
##########
File path:
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
##########
@@ -78,12 +88,27 @@ public static void main(String... args) {
jobReconciler,
sessionReconciler);
- FlinkControllerConfig controllerConfig = new
FlinkControllerConfig(controller);
+ controllerConfig = new FlinkControllerConfig(controller);
controller.setControllerConfig(controllerConfig);
controllerConfig.setConfigurationService(configurationService);
+ }
+ public void run() {
operator.register(controller, controllerConfig);
operator.installShutdownHook();
operator.start();
}
+
+ protected ConfigurationService getConfigurationService() {
+ return new
ConfigurationServiceDecorator(DefaultConfigurationService.instance()) {
+ @Override
+ public ExecutorService getExecutorService() {
+ return Executors.newCachedThreadPool();
Review comment:
Should this always be unlimited size? Maybe we should make this
configurable, what do you think
##########
File path:
flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/FlinkOperatorTest.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.flink.kubernetes.operator;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.IntStream;
+
+/** @link FlinkOperator unit tests. */
+public class FlinkOperatorTest {
+
+ @Test
+ public void testExecutorServiceCanRun100ThreadsParallel() throws Exception
{
Review comment:
Seems like the test is testing that cached executor service works that
it should. This feels pretty unnecessary, it would be enough I think to assert
that the executor service that we get back from the operator is the expected
type/config.
Even better would be to actually start the operator and try to submit
multiple different deployment (different names) in parallel and see that it
handles it.
##########
File path:
flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/FlinkOperator.java
##########
@@ -27,33 +27,43 @@
import org.apache.flink.kubernetes.operator.reconciler.SessionReconciler;
import org.apache.flink.kubernetes.operator.service.FlinkService;
import org.apache.flink.kubernetes.operator.utils.FlinkUtils;
+import
org.apache.flink.kubernetes.operator.utils.javaoperatorsdk.ConfigurationServiceDecorator;
import
org.apache.flink.kubernetes.operator.validation.DefaultDeploymentValidator;
import
org.apache.flink.kubernetes.operator.validation.FlinkDeploymentValidator;
import io.fabric8.kubernetes.client.DefaultKubernetesClient;
+import io.fabric8.kubernetes.client.KubernetesClient;
import io.javaoperatorsdk.operator.Operator;
+import io.javaoperatorsdk.operator.api.config.ConfigurationService;
import io.javaoperatorsdk.operator.config.runtime.DefaultConfigurationService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
/** Main Class for Flink native k8s operator. */
public class FlinkOperator {
private static final Logger LOG =
LoggerFactory.getLogger(FlinkOperator.class);
- public static void main(String... args) {
+ final Operator operator;
Review comment:
let's keep this private final and add a protected getter with the
@VisibleForTesting annotation
--
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]