You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@heron.apache.org by sa...@apache.org on 2022/07/19 17:38:35 UTC

[incubator-heron] branch saadurrahman/3846-Refactoring-K8s-Shim-dev created (now 73c5223c83b)

This is an automated email from the ASF dual-hosted git repository.

saadurrahman pushed a change to branch saadurrahman/3846-Refactoring-K8s-Shim-dev
in repository https://gitbox.apache.org/repos/asf/incubator-heron.git


      at 73c5223c83b [StatefulSet] groundwork for the Stateful Set factory.

This branch includes the following new commits:

     new 0fe52169397 [K8s] created the K8s shim
     new 73c5223c83b [StatefulSet] groundwork for the Stateful Set factory.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[incubator-heron] 01/02: [K8s] created the K8s shim

Posted by sa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

saadurrahman pushed a commit to branch saadurrahman/3846-Refactoring-K8s-Shim-dev
in repository https://gitbox.apache.org/repos/asf/incubator-heron.git

commit 0fe521693974fdf638bdf91213f6f30a57584b80
Author: Saad Ur Rahman <sa...@apache.org>
AuthorDate: Tue Jul 19 13:37:42 2022 -0400

    [K8s] created the K8s shim
    
    Refactored the V1Controller to KubernetesShim.
---
 .../scheduler/kubernetes/KubernetesContext.java    |  2 +-
 .../scheduler/kubernetes/KubernetesScheduler.java  |  2 +-
 .../{V1Controller.java => KubernetesShim.java}     |  6 +--
 .../scheduler/kubernetes/KubernetesUtils.java      |  2 +-
 heron/schedulers/tests/java/BUILD                  |  2 +-
 ...ControllerTest.java => KubernetesShimTest.java} | 43 +++++++++++-----------
 .../scheduler/kubernetes/KubernetesUtilsTest.java  |  4 +-
 .../heron/scheduler/kubernetes/VolumesTests.java   |  2 +-
 8 files changed, 32 insertions(+), 31 deletions(-)

diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesContext.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesContext.java
index 8d9a8795070..3d424ce5a89 100644
--- a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesContext.java
+++ b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesContext.java
@@ -219,7 +219,7 @@ public final class KubernetesContext extends Context {
   @VisibleForTesting
   protected static Map<String, Map<KubernetesConstants.VolumeConfigKeys, String>>
       getVolumeConfigs(final Config config, final String prefix, final boolean isExecutor) {
-    final Logger LOG = Logger.getLogger(V1Controller.class.getName());
+    final Logger LOG = Logger.getLogger(KubernetesShim.class.getName());
 
     final String prefixKey = String.format(prefix,
         isExecutor ? KubernetesConstants.EXECUTOR_NAME : KubernetesConstants.MANAGER_NAME);
diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java
index c35c87df7d5..abc35c77397 100644
--- a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java
+++ b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesScheduler.java
@@ -50,7 +50,7 @@ public class KubernetesScheduler implements IScheduler, IScalable {
   private UpdateTopologyManager updateTopologyManager;
 
   protected KubernetesController getController() {
-    return new V1Controller(configuration, runtimeConfiguration);
+    return new KubernetesShim(configuration, runtimeConfiguration);
   }
 
   @Override
diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/V1Controller.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesShim.java
similarity index 99%
rename from heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/V1Controller.java
rename to heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesShim.java
index 1cd85e5ad6f..ca339b8320a 100644
--- a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/V1Controller.java
+++ b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesShim.java
@@ -84,10 +84,10 @@ import okhttp3.Response;
 
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
 
-public class V1Controller extends KubernetesController {
+public class KubernetesShim extends KubernetesController {
 
   private static final Logger LOG =
-      Logger.getLogger(V1Controller.class.getName());
+      Logger.getLogger(KubernetesShim.class.getName());
 
   private static final String ENV_SHARD_ID = "SHARD_ID";
 
@@ -101,7 +101,7 @@ public class V1Controller extends KubernetesController {
    * @param configuration <code>topology</code> configurations.
    * @param runtimeConfiguration Kubernetes runtime configurations.
    */
-  V1Controller(Config configuration, Config runtimeConfiguration) {
+  KubernetesShim(Config configuration, Config runtimeConfiguration) {
     super(configuration, runtimeConfiguration);
 
     isPodTemplateDisabled = KubernetesContext.getPodTemplateDisabled(configuration);
diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesUtils.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesUtils.java
index 709662a9f30..db6fd706d24 100644
--- a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesUtils.java
+++ b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/KubernetesUtils.java
@@ -96,7 +96,7 @@ final class KubernetesUtils {
   }
 
   static class V1ControllerUtils<T> {
-    private static final Logger LOG = Logger.getLogger(V1Controller.class.getName());
+    private static final Logger LOG = Logger.getLogger(KubernetesShim.class.getName());
 
     /**
      * Merge two lists by keeping all values in the <code>primaryList</code> and de-duplicating values in
diff --git a/heron/schedulers/tests/java/BUILD b/heron/schedulers/tests/java/BUILD
index 3ad9ffedf36..0fb732314b2 100644
--- a/heron/schedulers/tests/java/BUILD
+++ b/heron/schedulers/tests/java/BUILD
@@ -205,7 +205,7 @@ java_tests(
         "org.apache.heron.scheduler.kubernetes.KubernetesLauncherTest",
         "org.apache.heron.scheduler.kubernetes.VolumesTests",
         "org.apache.heron.scheduler.kubernetes.KubernetesContextTest",
-        "org.apache.heron.scheduler.kubernetes.V1ControllerTest",
+        "org.apache.heron.scheduler.kubernetes.KubernetesShimTest",
         "org.apache.heron.scheduler.kubernetes.KubernetesUtilsTest",
     ],
     runtime_deps = [":kubernetes-tests"],
diff --git a/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/V1ControllerTest.java b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/KubernetesShimTest.java
similarity index 97%
rename from heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/V1ControllerTest.java
rename to heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/KubernetesShimTest.java
index a039ca980e5..04b57dcfc47 100644
--- a/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/V1ControllerTest.java
+++ b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/KubernetesShimTest.java
@@ -71,7 +71,7 @@ import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doReturn;
 
 @RunWith(MockitoJUnitRunner.class)
-public class V1ControllerTest {
+public class KubernetesShimTest {
 
   private static final String TOPOLOGY_NAME = "topology-name";
   private static final String CONFIGMAP_NAME = "CONFIG-MAP-NAME";
@@ -124,19 +124,19 @@ public class V1ControllerTest {
       .build();
 
   @Spy
-  private final V1Controller v1ControllerWithPodTemplate =
-      new V1Controller(CONFIG_WITH_POD_TEMPLATE, RUNTIME);
+  private final KubernetesShim v1ControllerWithPodTemplate =
+      new KubernetesShim(CONFIG_WITH_POD_TEMPLATE, RUNTIME);
 
   @Spy
-  private final V1Controller v1ControllerPodTemplate =
-      new V1Controller(configDisabledPodTemplate, RUNTIME);
+  private final KubernetesShim v1ControllerPodTemplate =
+      new KubernetesShim(configDisabledPodTemplate, RUNTIME);
 
   @Rule
   public final ExpectedException expectedException = ExpectedException.none();
 
   @Test
   public void testLoadPodFromTemplateDefault() {
-    final V1Controller v1ControllerNoPodTemplate = new V1Controller(CONFIG, RUNTIME);
+    final KubernetesShim v1ControllerNoPodTemplate = new KubernetesShim(CONFIG, RUNTIME);
     final V1PodTemplateSpec defaultPodSpec = new V1PodTemplateSpec();
 
     final V1PodTemplateSpec podSpecExecutor = v1ControllerNoPodTemplate.loadPodFromTemplate(true);
@@ -237,7 +237,7 @@ public class V1ControllerTest {
         .build();
 
     // Test case container.
-    // Input: ConfigMap to setup mock V1Controller, Boolean flag for executor/manager switch.
+    // Input: ConfigMap to setup mock KubernetesShim, Boolean flag for executor/manager switch.
     // Output: The expected error message.
     final List<TestTuple<Pair<V1ConfigMap, Boolean>, String>> testCases = new LinkedList<>();
     testCases.add(new TestTuple<>("Executor invalid Pod Template",
@@ -314,7 +314,7 @@ public class V1ControllerTest {
         .build();
 
     // Test case container.
-    // Input: ConfigMap to setup mock V1Controller, Boolean flag for executor/manager switch.
+    // Input: ConfigMap to setup mock KubernetesShim, Boolean flag for executor/manager switch.
     // Output: The expected Pod template as a string.
     final List<TestTuple<Pair<V1ConfigMap, Boolean>, String>> testCases = new LinkedList<>();
     testCases.add(new TestTuple<>("Executor valid Pod Template",
@@ -357,7 +357,7 @@ public class V1ControllerTest {
 
 
     // Test case container.
-    // Input: ConfigMap to setup mock V1Controller, Boolean flag for executor/manager switch.
+    // Input: ConfigMap to setup mock KubernetesShim, Boolean flag for executor/manager switch.
     // Output: The expected Pod template as a string.
     final List<TestTuple<Pair<V1ConfigMap, Boolean>, String>> testCases = new LinkedList<>();
     testCases.add(new TestTuple<>("Executor invalid Pod Template",
@@ -409,7 +409,7 @@ public class V1ControllerTest {
     final Config testConfig = Config.newBuilder()
         .put(POD_TEMPLATE_LOCATION_EXECUTOR, CONFIGMAP_POD_TEMPLATE_NAME)
         .build();
-    final V1Controller v1Controller = new V1Controller(testConfig, RUNTIME);
+    final KubernetesShim v1Controller = new KubernetesShim(testConfig, RUNTIME);
     final Pair<String, String> expected = new Pair<>(CONFIGMAP_NAME, POD_TEMPLATE_NAME);
 
     // Correct parsing
@@ -422,7 +422,7 @@ public class V1ControllerTest {
     expectedException.expect(TopologySubmissionException.class);
     final Config testConfig = Config.newBuilder()
         .put(POD_TEMPLATE_LOCATION_EXECUTOR, ".POD-TEMPLATE-NAME").build();
-    V1Controller v1Controller = new V1Controller(testConfig, RUNTIME);
+    KubernetesShim v1Controller = new KubernetesShim(testConfig, RUNTIME);
     v1Controller.getPodTemplateLocation(true);
   }
 
@@ -431,7 +431,7 @@ public class V1ControllerTest {
     expectedException.expect(TopologySubmissionException.class);
     final Config testConfig = Config.newBuilder()
         .put(POD_TEMPLATE_LOCATION_EXECUTOR, "CONFIGMAP-NAME.").build();
-    V1Controller v1Controller = new V1Controller(testConfig, RUNTIME);
+    KubernetesShim v1Controller = new KubernetesShim(testConfig, RUNTIME);
     v1Controller.getPodTemplateLocation(true);
   }
 
@@ -440,7 +440,7 @@ public class V1ControllerTest {
     expectedException.expect(TopologySubmissionException.class);
     final Config testConfig = Config.newBuilder()
         .put(POD_TEMPLATE_LOCATION_EXECUTOR, "CONFIGMAP-NAMEPOD-TEMPLATE-NAME").build();
-    V1Controller v1Controller = new V1Controller(testConfig, RUNTIME);
+    KubernetesShim v1Controller = new KubernetesShim(testConfig, RUNTIME);
     v1Controller.getPodTemplateLocation(true);
   }
 
@@ -450,9 +450,9 @@ public class V1ControllerTest {
     final int portNumberkept = 1111;
     final int numInstances = 3;
     final List<V1ContainerPort> expectedPortsBase =
-        Collections.unmodifiableList(V1Controller.getExecutorPorts());
+        Collections.unmodifiableList(KubernetesShim.getExecutorPorts());
     final List<V1ContainerPort> debugPorts =
-        Collections.unmodifiableList(V1Controller.getDebuggingPorts(numInstances));
+        Collections.unmodifiableList(KubernetesShim.getDebuggingPorts(numInstances));
     final List<V1ContainerPort> inputPortsBase = Collections.unmodifiableList(
         Arrays.asList(
             new V1ContainerPort()
@@ -509,7 +509,7 @@ public class V1ControllerTest {
   @Test
   public void testConfigureContainerEnvVars() {
     final List<V1EnvVar> heronEnvVars =
-        Collections.unmodifiableList(V1Controller.getExecutorEnvVars());
+        Collections.unmodifiableList(KubernetesShim.getExecutorEnvVars());
     final V1EnvVar additionEnvVar = new V1EnvVar()
         .name("env-variable-to-be-kept")
         .valueFrom(new V1EnvVarSource()
@@ -686,7 +686,7 @@ public class V1ControllerTest {
         .put(KubernetesContext.KUBERNETES_CONTAINER_VOLUME_MOUNT_NAME, pathNameDefault)
         .put(KubernetesContext.KUBERNETES_CONTAINER_VOLUME_MOUNT_PATH, pathDefault)
         .build();
-    final V1Controller controllerWithMounts = new V1Controller(configWithVolumes, RUNTIME);
+    final KubernetesShim controllerWithMounts = new KubernetesShim(configWithVolumes, RUNTIME);
     final V1VolumeMount volumeDefault = new V1VolumeMountBuilder()
         .withName(pathNameDefault)
         .withMountPath(pathDefault)
@@ -707,7 +707,8 @@ public class V1ControllerTest {
     );
 
     // No Volume Mounts set.
-    V1Controller controllerDoNotSetMounts = new V1Controller(Config.newBuilder().build(), RUNTIME);
+    KubernetesShim controllerDoNotSetMounts =
+        new KubernetesShim(Config.newBuilder().build(), RUNTIME);
     V1Container containerNoSetMounts = new V1Container();
     controllerDoNotSetMounts.mountVolumeIfPresent(containerNoSetMounts);
     Assert.assertNull(containerNoSetMounts.getVolumeMounts());
@@ -743,7 +744,7 @@ public class V1ControllerTest {
         .effect("Some Effect")
         .tolerationSeconds(5L);
     final List<V1Toleration> expectedTolerationBase =
-        Collections.unmodifiableList(V1Controller.getTolerations());
+        Collections.unmodifiableList(KubernetesShim.getTolerations());
     final List<V1Toleration> inputTolerationsBase = Collections.unmodifiableList(
         Arrays.asList(
             new V1Toleration()
@@ -838,7 +839,7 @@ public class V1ControllerTest {
     final V1PersistentVolumeClaim claimOne = new V1PersistentVolumeClaimBuilder()
         .withNewMetadata()
           .withName(volumeNameOne)
-          .withLabels(V1Controller.getPersistentVolumeClaimLabels(topologyName))
+          .withLabels(KubernetesShim.getPersistentVolumeClaimLabels(topologyName))
         .endMetadata()
         .withNewSpec()
           .withStorageClassName(storageClassName)
@@ -853,7 +854,7 @@ public class V1ControllerTest {
     final V1PersistentVolumeClaim claimStatic = new V1PersistentVolumeClaimBuilder()
         .withNewMetadata()
           .withName(volumeNameStatic)
-          .withLabels(V1Controller.getPersistentVolumeClaimLabels(topologyName))
+          .withLabels(KubernetesShim.getPersistentVolumeClaimLabels(topologyName))
         .endMetadata()
         .withNewSpec()
           .withStorageClassName("")
diff --git a/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/KubernetesUtilsTest.java b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/KubernetesUtilsTest.java
index ddc1f692dc4..48ff6aab01c 100644
--- a/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/KubernetesUtilsTest.java
+++ b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/KubernetesUtilsTest.java
@@ -43,14 +43,14 @@ public class KubernetesUtilsTest {
   public void testMergeListsDedupe() {
     final String description = "Pod Template Environment Variables";
     final List<V1EnvVar> heronEnvVars =
-        Collections.unmodifiableList(V1Controller.getExecutorEnvVars());
+        Collections.unmodifiableList(KubernetesShim.getExecutorEnvVars());
     final V1EnvVar additionEnvVar = new V1EnvVar()
         .name("env-variable-to-be-kept")
         .valueFrom(new V1EnvVarSource()
             .fieldRef(new V1ObjectFieldSelector()
                 .fieldPath("env-variable-was-kept")));
     final List<V1EnvVar> expectedEnvVars = Collections.unmodifiableList(
-        new LinkedList<V1EnvVar>(V1Controller.getExecutorEnvVars()) {{
+        new LinkedList<V1EnvVar>(KubernetesShim.getExecutorEnvVars()) {{
           add(additionEnvVar);
         }});
     final List<V1EnvVar> inputEnvVars = Arrays.asList(
diff --git a/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/VolumesTests.java b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/VolumesTests.java
index 22d3088b598..9becf8b8d7b 100644
--- a/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/VolumesTests.java
+++ b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/VolumesTests.java
@@ -244,7 +244,7 @@ public class VolumesTests {
     final String volumeMode = "VolumeMode";
     final String path = "/path/to/mount/";
     final String subPath = "/sub/path/to/mount/";
-    final Map<String, String> labels = V1Controller.getPersistentVolumeClaimLabels(topologyName);
+    final Map<String, String> labels = KubernetesShim.getPersistentVolumeClaimLabels(topologyName);
 
     final Map<KubernetesConstants.VolumeConfigKeys, String> volOneConfig =
         new HashMap<KubernetesConstants.VolumeConfigKeys, String>() {


[incubator-heron] 02/02: [StatefulSet] groundwork for the Stateful Set factory.

Posted by sa...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

saadurrahman pushed a commit to branch saadurrahman/3846-Refactoring-K8s-Shim-dev
in repository https://gitbox.apache.org/repos/asf/incubator-heron.git

commit 73c5223c83b910f87e4e6e5ad6cb07a3f0ea9a3f
Author: Saad Ur Rahman <sa...@apache.org>
AuthorDate: Tue Jul 19 13:38:22 2022 -0400

    [StatefulSet] groundwork for the Stateful Set factory.
---
 .../heron/scheduler/kubernetes/StatefulSet.java    | 40 ++++++++++++++++++++++
 heron/schedulers/tests/java/BUILD                  |  1 +
 .../scheduler/kubernetes/StatefulSetTest.java      | 23 +++++++++++++
 3 files changed, 64 insertions(+)

diff --git a/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/StatefulSet.java b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/StatefulSet.java
new file mode 100644
index 00000000000..ee763e97aea
--- /dev/null
+++ b/heron/schedulers/src/java/org/apache/heron/scheduler/kubernetes/StatefulSet.java
@@ -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.
+ */
+
+package org.apache.heron.scheduler.kubernetes;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.heron.spi.packing.Resource;
+
+import io.kubernetes.client.openapi.models.V1StatefulSet;
+
+final class StatefulSet {
+  private final Map<String, StatefulSetFactory> statefulsets = new HashMap<>();
+
+  public enum Type {
+    Executor,
+    Manager
+  }
+
+  interface StatefulSetFactory {
+    V1StatefulSet create(Type type, Resource containerResources, int numberOfInstances);
+  }
+}
diff --git a/heron/schedulers/tests/java/BUILD b/heron/schedulers/tests/java/BUILD
index 0fb732314b2..c8eac176c74 100644
--- a/heron/schedulers/tests/java/BUILD
+++ b/heron/schedulers/tests/java/BUILD
@@ -206,6 +206,7 @@ java_tests(
         "org.apache.heron.scheduler.kubernetes.VolumesTests",
         "org.apache.heron.scheduler.kubernetes.KubernetesContextTest",
         "org.apache.heron.scheduler.kubernetes.KubernetesShimTest",
+        "org.apache.heron.scheduler.kubernetes.StatefulSetTest",
         "org.apache.heron.scheduler.kubernetes.KubernetesUtilsTest",
     ],
     runtime_deps = [":kubernetes-tests"],
diff --git a/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/StatefulSetTest.java b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/StatefulSetTest.java
new file mode 100644
index 00000000000..5dedaa97178
--- /dev/null
+++ b/heron/schedulers/tests/java/org/apache/heron/scheduler/kubernetes/StatefulSetTest.java
@@ -0,0 +1,23 @@
+/**
+ * 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.heron.scheduler.kubernetes;
+
+public class StatefulSetTest {
+}