You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by GitBox <gi...@apache.org> on 2022/08/18 21:13:10 UTC

[GitHub] [samza] james-deee opened a new pull request, #1628: Get Samza to work with Java 11 (still works with Java 8)

james-deee opened a new pull request, #1628:
URL: https://github.com/apache/samza/pull/1628

   This has 2 specific purposes.
   
   - Upgrade the Yarn Version to a Java 11 compatible version
   - Make everything compile when building with Java 11.
   - 
   I have tested that both Java 8 and Java 11 compile and all tests pass from the above.


-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee commented on a diff in pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
james-deee commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r976687067


##########
samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.job.yarn
+
+import java.net.URL
+import java.nio.ByteBuffer
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler
+import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.apache.samza.SamzaException
+import org.apache.samza.clustermanager.SamzaApplicationState
+import org.apache.samza.clustermanager.SamzaApplicationState.SamzaAppStatus
+import org.apache.samza.coordinator.JobModelManager
+import org.junit.Assert._
+import org.junit.Test
+import org.mockito.{ArgumentCaptor, Mockito}
+
+class TestSamzaYarnAppMasterLifecycle {
+  private def YARN_CONTAINER_ID = "container_123_123_123"
+  private def YARN_CONTAINER_HOST = "host"
+  private def YARN_CONTAINER_MEM = 1024
+  private def YARN_CONTAINER_VCORE = 1
+  val coordinator = new JobModelManager(null, null)
+  val amClient = new AMRMClientAsyncImpl[ContainerRequest](1, Mockito.mock(classOf[CallbackHandler])) {
+    var host = ""
+    var port = 0
+    var status: FinalApplicationStatus = null
+    override def registerApplicationMaster(appHostName: String, appHostPort: Int, appTrackingUrl: String): RegisterApplicationMasterResponse = {
+      this.host = appHostName
+      this.port = appHostPort
+      new RegisterApplicationMasterResponse {
+        override def setApplicationACLs(map: java.util.Map[ApplicationAccessType, String]): Unit = ()
+        override def getApplicationACLs = null
+        override def setResourceTypes(types: java.util.List[org.apache.hadoop.yarn.api.records.ResourceTypeInfo]): Unit = ()
+        override def getResourceTypes = null
+        override def setMaximumResourceCapability(r: Resource): Unit = ()
+        override def getMaximumResourceCapability = new Resource {
+          def getMemory = 512
+          def getVirtualCores = 2
+          def setMemory(memory: Int) {}
+          def setVirtualCores(vCores: Int) {}
+          override def compareTo(o: Resource) = 0
+        }
+        override def getClientToAMTokenMasterKey = null
+        override def setClientToAMTokenMasterKey(buffer: ByteBuffer) {}
+        // to test AM high availability - return a running container from previous attempt
+        val prevAttemptCotainers = new java.util.ArrayList[Container]()
+        prevAttemptCotainers.add(getMockContainer)
+        override def getContainersFromPreviousAttempts(): java.util.List[Container] = prevAttemptCotainers
+        override def getNMTokensFromPreviousAttempts(): java.util.List[NMToken] = java.util.Collections.emptyList[NMToken]
+        override def getQueue(): String = null
+        override def setContainersFromPreviousAttempts(containers: java.util.List[Container]): Unit = Unit
+        override def setNMTokensFromPreviousAttempts(nmTokens: java.util.List[NMToken]): Unit = Unit
+        override def setQueue(queue: String): Unit = Unit
+
+        override def setSchedulerResourceTypes(types: java.util.EnumSet[SchedulerResourceTypes]): Unit = {}
+        override def getSchedulerResourceTypes: java.util.EnumSet[SchedulerResourceTypes] = null
+        override def getResourceProfiles(): java.util.Map[String,org.apache.hadoop.yarn.api.records.Resource] = ???

Review Comment:
   The `getResourceProfiles` and `setResourceProfiles` are 2 required overrides for Yarn3. Yarn2 does NOT have these overrides.



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] nickpan47 merged pull request #1628: SAMZA-2757: Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
nickpan47 merged PR #1628:
URL: https://github.com/apache/samza/pull/1628


-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee closed pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
james-deee closed pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11
URL: https://github.com/apache/samza/pull/1628


-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] nickpan47 commented on a diff in pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
nickpan47 commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r983075086


##########
gradle.properties:
##########
@@ -16,10 +16,14 @@
 # under the License.
 group=org.apache.samza
 version=1.7.0-SNAPSHOT
+# These 2 are ones that you can override using properties, like:
+# -PscalaSuffix=2.12 -PyarnVersion=2.10.1
 scalaSuffix=2.12
+yarnVersion=2.10.1
+yarn3Version=3.3.4

Review Comment:
   Add a comment to state that this is to support JDK11 runtime.



##########
samza-log4j2/src/main/java/org/apache/samza/logging/log4j2/StreamAppender.java:
##########
@@ -73,7 +73,7 @@ public class StreamAppender extends AbstractAppender {
   private final BlockingQueue<EncodedLogEvent> logQueue = new LinkedBlockingQueue<>(DEFAULT_QUEUE_SIZE);
 
   private SystemStream systemStream = null;
-  private SystemProducer systemProducer = null;
+  protected SystemProducer systemProducer = null;

Review Comment:
   Why are we changing this to "protected"? I would recommend keeping this PR just for JDK11 changes. If this is needed for other reasons, please open a separate PR.
   
   P.S. it seems PR #1627 has already made the change.



##########
samza-shell/src/main/bash/run-class.sh:
##########
@@ -153,7 +153,7 @@ fi
 [[ $JAVA_OPTS != *-Xmx* ]] && JAVA_OPTS="$JAVA_OPTS -Xmx768M"
 
 # Check if the GC related flags are specified. If not - add the respective flags to JVM_OPTS.
-[[ $JAVA_OPTS != *PrintGCDateStamps* && $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -XX:+PrintGCDateStamps -Xloggc:$SAMZA_LOG_DIR/gc.log"
+[[ $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -Xloggc:$SAMZA_LOG_DIR/gc.log"

Review Comment:
   Can we make it conditional? i.e. if JAVA_HOME is 8, use this, if JAVA_HOME is 11, don't use this?



##########
README.md:
##########
@@ -28,9 +28,12 @@ After the bootstrap script has completed, the regular gradlew instructions below
 
 #### Scala and YARN
 
-Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.6.1, by default. Use the -PscalaSuffix switches to change Scala versions. Samza supports building Scala with 2.11 and 2.12.
+Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.10.1 or 3.3.4. Scala 2.12 and Yarn 2.10.1 are used by default. Use the -PscalaSuffix and -PyarnVersion switches to change Scala or Yarn versions. Samza supports building Scala with 2.11 and 2.12.

Review Comment:
   I think that we should still keep the original description, w/ default to 2.10.1. The separate build commands below are also not needed.
   
   We may want to add a section separately regarding to JDK11 runtime support, i.e. stating that Samza currently is still built w/ Java8 but support Java11 runtime, w/ samza-yarn3 built on YARN 3.3.4.



##########
samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java:
##########
@@ -60,22 +60,22 @@ public class TestYarnFaultDomainManager {
 
   private final NodeReport nodeReport1 = createNodeReport(hostName1, 1, NodeState.RUNNING, "httpAddress1",
           rackName1, 1, 1, 2, 1, 2,
-          "", 60L, null);
+          "", 60L);

Review Comment:
   Is this change for YARN 3.3.4? I would suggest leave it out since we are not changing samza-yarn module for YARN 3.3.4.



##########
gradle.properties:
##########
@@ -16,10 +16,14 @@
 # under the License.
 group=org.apache.samza
 version=1.7.0-SNAPSHOT
+# These 2 are ones that you can override using properties, like:
+# -PscalaSuffix=2.12 -PyarnVersion=2.10.1
 scalaSuffix=2.12
+yarnVersion=2.10.1
+yarn3Version=3.3.4
 
 # after changing this value, run `$ ./gradlew wrapper` and commit the resulting changed files
-gradleVersion=5.2.1
+gradleVersion=6.9.2

Review Comment:
   Is this strictly needed for JDK11 runtime support? @mynameborat can you comment on that?



##########
samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java:
##########
@@ -182,9 +182,17 @@ private void assertFaultDomainEquals(FaultDomain faultDomain1, FaultDomain fault
 
   private NodeReport createNodeReport(String host, int port, NodeState nodeState, String httpAddress, String rackName,
                                       int memoryUsed, int vcoresUsed, int totalMemory, int totalVcores, int numContainers,
-                                      String healthReport, long lastHealthReportTime, Set<String> nodeLabels) {
-    return NodeReport.newInstance(NodeId.newInstance(host, port), nodeState, httpAddress, rackName,
-            Resource.newInstance(memoryUsed, vcoresUsed), Resource.newInstance(totalMemory, totalVcores), numContainers,
-            healthReport, lastHealthReportTime, nodeLabels);
+                                      String healthReport, long lastHealthReportTime) {
+    return NodeReport.newInstance(

Review Comment:
   Same comment here: is this change for YARN 3.3.4? If not, let's leave changes not related to JDK11 runtime support out of this PR.



##########
README.md:
##########
@@ -28,9 +28,12 @@ After the bootstrap script has completed, the regular gradlew instructions below
 
 #### Scala and YARN
 
-Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.6.1, by default. Use the -PscalaSuffix switches to change Scala versions. Samza supports building Scala with 2.11 and 2.12.
+Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.10.1 or 3.3.4. Scala 2.12 and Yarn 2.10.1 are used by default. Use the -PscalaSuffix and -PyarnVersion switches to change Scala or Yarn versions. Samza supports building Scala with 2.11 and 2.12.

Review Comment:
   Also, please list the modules that support jdk11 runtime in the above section (i.e. not samza-yarn, samza-shell, samza-test, and samza-hdfs, since those modules are still on YARN 2.10.1).



##########
README.md:
##########
@@ -28,9 +28,12 @@ After the bootstrap script has completed, the regular gradlew instructions below
 
 #### Scala and YARN
 
-Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.6.1, by default. Use the -PscalaSuffix switches to change Scala versions. Samza supports building Scala with 2.11 and 2.12.
+Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.10.1 or 3.3.4. Scala 2.12 and Yarn 2.10.1 are used by default. Use the -PscalaSuffix and -PyarnVersion switches to change Scala or Yarn versions. Samza supports building Scala with 2.11 and 2.12.
 
-    ./gradlew -PscalaSuffix=2.11 clean build
+    ./gradlew -PscalaSuffix=2.12 -PyarnVersion=2.10.1 clean build

Review Comment:
   The build command should still stay the same.



##########
samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala:
##########
@@ -78,6 +78,8 @@ class TestSamzaYarnAppMasterLifecycle {
 
         override def setSchedulerResourceTypes(types: java.util.EnumSet[SchedulerResourceTypes]): Unit = {}
         override def getSchedulerResourceTypes: java.util.EnumSet[SchedulerResourceTypes] = null
+        //override def getResourceProfiles(): java.util.Map[String,org.apache.hadoop.yarn.api.records.Resource] = ???

Review Comment:
   Please remove the commented lines here.



##########
samza-log4j2/src/main/java/org/apache/samza/logging/log4j2/StreamAppender.java:
##########
@@ -416,7 +416,7 @@ private void startTransferThread() {
    * Helper method to send a serialized log-event to the systemProducer, and increment respective methods.
    * @param logQueueEntry the serialized log-event to be sent to the systemProducer
    */
-  private void sendEventToSystemProducer(EncodedLogEvent logQueueEntry) {
+  protected void sendEventToSystemProducer(EncodedLogEvent logQueueEntry) {

Review Comment:
   Same here. This is in PR#1627. Let's keep this PR only on JDK11 runtime support.



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee commented on a diff in pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
james-deee commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r989180206


##########
build.gradle:
##########
@@ -590,6 +660,73 @@ project(":samza-shell") {
   }
 }
 
+project(":samza-shell-yarn3") {
+  apply plugin: 'java'
+
+  configurations {
+    gradleShell
+  }
+
+  dependencies {
+    gradleShell project(":samza-core_$scalaSuffix")
+    gradleShell project(":samza-kafka_$scalaSuffix")
+    gradleShell project(":samza-test_$scalaSuffix")
+    gradleShell project(":samza-yarn3_$scalaSuffix")
+  }
+
+  task shellTarGz(type: Tar) {
+    compression = Compression.GZIP
+    classifier = 'dist'
+    from 'src/main/bash'

Review Comment:
   Oh jeez. what a dummy i am. Should probably make sure the actual distribution get's build :facepalm: .
   
   Thanks! I updated it.



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] nickpan47 commented on a diff in pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
nickpan47 commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r988552047


##########
build.gradle:
##########
@@ -590,6 +660,73 @@ project(":samza-shell") {
   }
 }
 
+project(":samza-shell-yarn3") {
+  apply plugin: 'java'
+
+  configurations {
+    gradleShell
+  }
+
+  dependencies {
+    gradleShell project(":samza-core_$scalaSuffix")
+    gradleShell project(":samza-kafka_$scalaSuffix")
+    gradleShell project(":samza-test_$scalaSuffix")
+    gradleShell project(":samza-yarn3_$scalaSuffix")
+  }
+
+  task shellTarGz(type: Tar) {
+    compression = Compression.GZIP
+    classifier = 'dist'
+    from 'src/main/bash'

Review Comment:
   Are you sure that this works? I pulled down the PR and built locally. Only found that the corresponding samza-shell-yarn3 build does not yield any tgz file as output in build/distribution, since there is no separate samza-shell-yarn3/src in the source repo.



##########
build.gradle:
##########
@@ -590,6 +660,73 @@ project(":samza-shell") {
   }
 }
 
+project(":samza-shell-yarn3") {
+  apply plugin: 'java'
+
+  configurations {
+    gradleShell
+  }
+
+  dependencies {
+    gradleShell project(":samza-core_$scalaSuffix")
+    gradleShell project(":samza-kafka_$scalaSuffix")
+    gradleShell project(":samza-test_$scalaSuffix")
+    gradleShell project(":samza-yarn3_$scalaSuffix")
+  }
+
+  task shellTarGz(type: Tar) {
+    compression = Compression.GZIP
+    classifier = 'dist'
+    from 'src/main/bash'

Review Comment:
   Made the following change here and the build command successfully generates the samza-shell-yarn3 .tgz file:
   
   `
   -    from 'src/main/bash'
   -    from 'src/main/resources'
   -    from 'src/main/visualizer'
   +    from(project(":samza-shell").file("./src/main/bash"))
   +    from(project(":samza-shell").file("./src/main/resources"))
   +    from(project(":samza-shell").file("./src/main/visualizer"))
   `



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee commented on a diff in pull request #1628: Get Samza to work with Java 11 (still works with Java 8)

Posted by GitBox <gi...@apache.org>.
james-deee commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r949607696


##########
samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala:
##########
@@ -366,16 +366,14 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
     val properties = new Properties()
 
     if (isStreamMode) {
-      properties.putAll(ImmutableMap.of(
-        "cleanup.policy", "compact",
-        "segment.bytes", String.valueOf(segmentBytes),
-        "max.message.bytes", String.valueOf(maxMessageBytes)))
+      properties.put("cleanup.policy", "compact")

Review Comment:
   The putAll method has an ambiguous issue: https://github.com/scala/bug/issues/10418
   
   So just moving to single line puts.



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee commented on a diff in pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
james-deee commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r986026649


##########
gradle.properties:
##########
@@ -16,10 +16,14 @@
 # under the License.
 group=org.apache.samza
 version=1.7.0-SNAPSHOT
+# These 2 are ones that you can override using properties, like:
+# -PscalaSuffix=2.12 -PyarnVersion=2.10.1
 scalaSuffix=2.12
+yarnVersion=2.10.1
+yarn3Version=3.3.4
 
 # after changing this value, run `$ ./gradlew wrapper` and commit the resulting changed files
-gradleVersion=5.2.1
+gradleVersion=6.9.2

Review Comment:
   No it is not strictly needed to support 11. But everything seems to build and run as expected, so that it was worth at least moving forward to be more up to date. :shrug: I'm ok reverting if needed, but things seem good here.



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee commented on pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
james-deee commented on PR #1628:
URL: https://github.com/apache/samza/pull/1628#issuecomment-1235813671

   @xinyuiscool Sorry to tag you, but I see you were the last one to merge something to this project. Would you be able to take a look at this PR, and the corresponding hello samza one? https://github.com/apache/samza-hello-samza/pull/87


-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee commented on a diff in pull request #1628: SAMZA-2757 : Make Samza Compatible with Java 11

Posted by GitBox <gi...@apache.org>.
james-deee commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r986061798


##########
samza-shell/src/main/bash/run-class.sh:
##########
@@ -153,7 +153,7 @@ fi
 [[ $JAVA_OPTS != *-Xmx* ]] && JAVA_OPTS="$JAVA_OPTS -Xmx768M"
 
 # Check if the GC related flags are specified. If not - add the respective flags to JVM_OPTS.
-[[ $JAVA_OPTS != *PrintGCDateStamps* && $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -XX:+PrintGCDateStamps -Xloggc:$SAMZA_LOG_DIR/gc.log"
+[[ $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -Xloggc:$SAMZA_LOG_DIR/gc.log"

Review Comment:
   Sure. Just added the conditional for it. 



##########
samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala:
##########
@@ -78,6 +78,8 @@ class TestSamzaYarnAppMasterLifecycle {
 
         override def setSchedulerResourceTypes(types: java.util.EnumSet[SchedulerResourceTypes]): Unit = {}
         override def getSchedulerResourceTypes: java.util.EnumSet[SchedulerResourceTypes] = null
+        //override def getResourceProfiles(): java.util.Map[String,org.apache.hadoop.yarn.api.records.Resource] = ???

Review Comment:
   done



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] james-deee commented on a diff in pull request #1628: Get Samza to work with Java 11 (still works with Java 8)

Posted by GitBox <gi...@apache.org>.
james-deee commented on code in PR #1628:
URL: https://github.com/apache/samza/pull/1628#discussion_r949607846


##########
samza-shell/src/main/bash/run-class.sh:
##########
@@ -153,7 +153,7 @@ fi
 [[ $JAVA_OPTS != *-Xmx* ]] && JAVA_OPTS="$JAVA_OPTS -Xmx768M"
 
 # Check if the GC related flags are specified. If not - add the respective flags to JVM_OPTS.
-[[ $JAVA_OPTS != *PrintGCDateStamps* && $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -XX:+PrintGCDateStamps -Xloggc:$SAMZA_LOG_DIR/gc.log"
+[[ $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -Xloggc:$SAMZA_LOG_DIR/gc.log"

Review Comment:
   PrintGCDateStamps is removed from Java11, so just removing it from here.



-- 
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: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org