[jira] [Reopened] (SAMZA-1482) Restart or fail Samza jobs in YARN when input topic partition changes
[ https://issues.apache.org/jira/browse/SAMZA-1482?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Pan (Data Infrastructure) reopened SAMZA-1482: - Still missing some config doc update. > Restart or fail Samza jobs in YARN when input topic partition changes > - > > Key: SAMZA-1482 > URL: https://issues.apache.org/jira/browse/SAMZA-1482 > Project: Samza > Issue Type: Bug >Reporter: Yi Pan (Data Infrastructure) >Assignee: Yi Pan (Data Infrastructure) > Fix For: 0.14.0 > > Original Estimate: 168h > Remaining Estimate: 168h > > Currently, after a Samza job is started, it works only on a set of fixed > input topic partitions at the start-up time. When input topic partitions are > expanded, we often lose the messages sent in the new partitions, until we > restart the job. > SAMZA-882 added a input stream partition count monitor inside the > JobCoordinator. This ticket is targeted to use this monitor metrics and > trigger the following actions in YARN: > # for stateless jobs, shutdown the JobCoordinator w/ UNDEFINED status code > s.t. YARN will restart the whole job > # for stateful jobs, shutdown the JobCoordinator w/ FAILED status code s.t. > YARN will stop the whole job -- This message was sent by Atlassian JIRA (v6.4.14#64029)
samza git commit: SAMZA-1482: Restart or fail Samza jobs in YARN when detecting changes…
Repository: samza Updated Branches: refs/heads/master 3750f5e24 -> 6fcf7f3f4 SAMZA-1482: Restart or fail Samza jobs in YARN when detecting changes⦠⦠in input topic partitions Some high-lights of the changes: - always instantiating StreamPartitionCountMonitor on all input system streams now -- it is debatable whether we want to include systems that do not implement the optimized ExtendedSystemAdmin interface. We may need to configure a long partition monitor interval for this case and the case where there are tons of input topics. (Pending perf test) - moved the instantiation of StreamPartitionCountMonitor out of JobModelManager and allow ClusterBasedJobCoordinator associate a callback method directly to the monitor - allow callbacks to set different application status code before throwing exception to shutdown the job Author: Yi Pan (Data Infrastructure)Reviewers: Jacob Maes , Jagadish Closes #351 from nickpan47/restart-on-partition-change and squashes the following commits: 8d04cd6 [Yi Pan (Data Infrastructure)] SAMZA-1482: restart or fail the job when input topic partition count changes ee3fa65 [Yi Pan (Data Infrastructure)] SAMZA-1482: Restart or fail Samza jobs in YARN when detecting changes in input topic partitions Project: http://git-wip-us.apache.org/repos/asf/samza/repo Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/6fcf7f3f Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/6fcf7f3f Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/6fcf7f3f Branch: refs/heads/master Commit: 6fcf7f3f4cbf8b9c6f69b292e3c1aaa239ab18d3 Parents: 3750f5e Author: Yi Pan (Data Infrastructure) Authored: Tue Nov 14 12:45:23 2017 -0800 Committer: Yi Pan (Data Infrastructure) Committed: Tue Nov 14 12:45:23 2017 -0800 -- .../apache/samza/PartitionChangeException.java | 31 .../ClusterBasedJobCoordinator.java | 108 +-- .../clustermanager/ContainerProcessManager.java | 26 ++- .../clustermanager/SamzaApplicationState.java | 7 +- .../StreamPartitionCountMonitor.java| 116 +++- .../org/apache/samza/config/JobConfig.scala | 2 - .../org/apache/samza/config/StorageConfig.scala | 8 + .../samza/coordinator/JobModelManager.scala | 49 ++--- .../main/scala/org/apache/samza/util/Util.scala | 1 - .../MockClusterResourceManagerFactory.java | 32 .../clustermanager/MockContainerListener.java | 1 + .../TestClusterBasedJobCoordinator.java | 108 +++ .../clustermanager/TestContainerAllocator.java | 12 +- .../TestContainerProcessManager.java| 22 +-- .../TestHostAwareContainerAllocator.java| 16 +- .../coordinator/JobModelManagerTestUtil.java| 4 +- .../samza/coordinator/TestJobModelManager.java | 24 +-- .../samza/storage/MockSystemConsumer.java | 59 -- .../apache/samza/storage/MockSystemFactory.java | 45 - .../samza/storage/TestStorageRecovery.java | 37 +--- .../apache/samza/system/MockSystemFactory.java | 181 +++ .../samza/coordinator/TestJobCoordinator.scala | 65 +-- .../TestStreamPartitionCountMonitor.scala | 82 - .../TestRangeSystemStreamPartitionMatcher.scala | 1 - .../TestRegexSystemStreamPartitionMatcher.scala | 1 - 25 files changed, 693 insertions(+), 345 deletions(-) -- http://git-wip-us.apache.org/repos/asf/samza/blob/6fcf7f3f/samza-core/src/main/java/org/apache/samza/PartitionChangeException.java -- diff --git a/samza-core/src/main/java/org/apache/samza/PartitionChangeException.java b/samza-core/src/main/java/org/apache/samza/PartitionChangeException.java new file mode 100644 index 000..4619dfa --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/PartitionChangeException.java @@ -0,0 +1,31 @@ +/* + * 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; + + +/** + *
[jira] [Commented] (SAMZA-1501) Validate operator IDs so that they don't contain special characters and spaces
[ https://issues.apache.org/jira/browse/SAMZA-1501?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16252150#comment-16252150 ] ASF GitHub Bot commented on SAMZA-1501: --- GitHub user prateekm opened a pull request: https://github.com/apache/samza/pull/359 SAMZA-1501: Validate operator IDs so that they don't contain special characters and spaces You can merge this pull request into a Git repository by running: $ git pull https://github.com/prateekm/samza operator-id-validation Alternatively you can review and apply these changes as the patch at: https://github.com/apache/samza/pull/359.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #359 commit f27d4b5be08ea2a5ec7b533185acc42687e5a706 Author: Prateek MaheshwariDate: 2017-11-14T20:42:30Z SAMZA-1501: Validate operator IDs so that they don't contain special characters and spaces > Validate operator IDs so that they don't contain special characters and spaces > -- > > Key: SAMZA-1501 > URL: https://issues.apache.org/jira/browse/SAMZA-1501 > Project: Samza > Issue Type: Task >Reporter: Prateek Maheshwari >Assignee: Prateek Maheshwari > > Since user provided operator IDs are also used as changelog stream names, we > should validate that they don't contain special characters and spaces. > If Kafka is used for the changelog topic, it'll fail with an exception like > the following: > {code} > 2017-11-14 11:45:14.408 [main] KafkaSystemAdmin [WARN] Failed to create topic > wikipedia-application-1-window-Tumbling window of WikipediaStats: > org.apache.kafka.common.errors.InvalidTopicException: topic name > wikipedia-application-1-window-Tumbling window of WikipediaStats is illegal, > contains a character other than ASCII alphanumerics, '.', '_' and '-'. > Retrying. > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1502) Make AllSspToSingleTaskGrouper work with Yarn and ZK JobCoordinator
Aditya created SAMZA-1502: - Summary: Make AllSspToSingleTaskGrouper work with Yarn and ZK JobCoordinator Key: SAMZA-1502 URL: https://issues.apache.org/jira/browse/SAMZA-1502 Project: Samza Issue Type: Bug Reporter: Aditya Currently, AllSspToSingleTaskGrouper works only with Passthrough JobCoordinator and this ticket is to extend the grouper to work with Yarn and ZK JobCoordinator. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Resolved] (SAMZA-1482) Restart or fail Samza jobs in YARN when input topic partition changes
[ https://issues.apache.org/jira/browse/SAMZA-1482?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yi Pan (Data Infrastructure) resolved SAMZA-1482. - Resolution: Fixed Fix Version/s: 0.14.0 > Restart or fail Samza jobs in YARN when input topic partition changes > - > > Key: SAMZA-1482 > URL: https://issues.apache.org/jira/browse/SAMZA-1482 > Project: Samza > Issue Type: Bug >Reporter: Yi Pan (Data Infrastructure) >Assignee: Yi Pan (Data Infrastructure) > Fix For: 0.14.0 > > Original Estimate: 168h > Remaining Estimate: 168h > > Currently, after a Samza job is started, it works only on a set of fixed > input topic partitions at the start-up time. When input topic partitions are > expanded, we often lose the messages sent in the new partitions, until we > restart the job. > SAMZA-882 added a input stream partition count monitor inside the > JobCoordinator. This ticket is targeted to use this monitor metrics and > trigger the following actions in YARN: > # for stateless jobs, shutdown the JobCoordinator w/ UNDEFINED status code > s.t. YARN will restart the whole job > # for stateful jobs, shutdown the JobCoordinator w/ FAILED status code s.t. > YARN will stop the whole job -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1501) Validate operator IDs so that they don't contain special characters and spaces
Prateek Maheshwari created SAMZA-1501: - Summary: Validate operator IDs so that they don't contain special characters and spaces Key: SAMZA-1501 URL: https://issues.apache.org/jira/browse/SAMZA-1501 Project: Samza Issue Type: Task Reporter: Prateek Maheshwari Assignee: Prateek Maheshwari Since user provided operator IDs are also used as changelog stream names, we should validate that they don't contain special characters and spaces. If Kafka is used for the changelog topic, it'll fail with an exception like the following: {code} 2017-11-14 11:45:14.408 [main] KafkaSystemAdmin [WARN] Failed to create topic wikipedia-application-1-window-Tumbling window of WikipediaStats: org.apache.kafka.common.errors.InvalidTopicException: topic name wikipedia-application-1-window-Tumbling window of WikipediaStats is illegal, contains a character other than ASCII alphanumerics, '.', '_' and '-'. Retrying. {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (SAMZA-1482) Restart or fail Samza jobs in YARN when input topic partition changes
[ https://issues.apache.org/jira/browse/SAMZA-1482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16252151#comment-16252151 ] ASF GitHub Bot commented on SAMZA-1482: --- Github user asfgit closed the pull request at: https://github.com/apache/samza/pull/351 > Restart or fail Samza jobs in YARN when input topic partition changes > - > > Key: SAMZA-1482 > URL: https://issues.apache.org/jira/browse/SAMZA-1482 > Project: Samza > Issue Type: Bug >Reporter: Yi Pan (Data Infrastructure) >Assignee: Yi Pan (Data Infrastructure) > Original Estimate: 168h > Remaining Estimate: 168h > > Currently, after a Samza job is started, it works only on a set of fixed > input topic partitions at the start-up time. When input topic partitions are > expanded, we often lose the messages sent in the new partitions, until we > restart the job. > SAMZA-882 added a input stream partition count monitor inside the > JobCoordinator. This ticket is targeted to use this monitor metrics and > trigger the following actions in YARN: > # for stateless jobs, shutdown the JobCoordinator w/ UNDEFINED status code > s.t. YARN will restart the whole job > # for stateful jobs, shutdown the JobCoordinator w/ FAILED status code s.t. > YARN will stop the whole job -- This message was sent by Atlassian JIRA (v6.4.14#64029)
samza git commit: SAMZA-1492: Add exception counter to LocalStoreMonitor.
Repository: samza Updated Branches: refs/heads/master d35355aa0 -> 26dd9c14c SAMZA-1492: Add exception counter to LocalStoreMonitor. Add storeGCExceptionCounter to LocalStoreMonitor(as a part of LocalStoreMonitorMetrics) to enable alerts setup in an production environment. Author: Shanthoosh VenkataramanReviewers: Jacob Maes Closes #357 from shanthoosh/local_store_monitor_metrics_1 Project: http://git-wip-us.apache.org/repos/asf/samza/repo Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/26dd9c14 Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/26dd9c14 Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/26dd9c14 Branch: refs/heads/master Commit: 26dd9c14c778862058ddaf8e866f8f12e77457d9 Parents: d35355a Author: Shanthoosh Venkataraman Authored: Tue Nov 14 07:56:13 2017 -0800 Committer: Jacob Maes Committed: Tue Nov 14 07:56:13 2017 -0800 -- .../main/java/org/apache/samza/monitor/LocalStoreMonitor.java| 1 + .../java/org/apache/samza/monitor/LocalStoreMonitorMetrics.java | 4 2 files changed, 5 insertions(+) -- http://git-wip-us.apache.org/repos/asf/samza/blob/26dd9c14/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitor.java -- diff --git a/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitor.java b/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitor.java index 8b25636..01b8ed7 100644 --- a/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitor.java +++ b/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitor.java @@ -104,6 +104,7 @@ public class LocalStoreMonitor implements Monitor { } } } catch (Exception ex) { +localStoreMonitorMetrics.failedStoreDeletionAttempts.inc(); if (!config.getIgnoreFailures()) { throw ex; } http://git-wip-us.apache.org/repos/asf/samza/blob/26dd9c14/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorMetrics.java -- diff --git a/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorMetrics.java b/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorMetrics.java index 204f3dd..0ff32fe 100644 --- a/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorMetrics.java +++ b/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorMetrics.java @@ -33,9 +33,13 @@ public class LocalStoreMonitorMetrics extends MetricsBase { /** Total disk space cleared by the LocalStoreMonitor. */ public final Counter diskSpaceFreedInBytes; + /** Total number of times task store deletions have been attempted and failed. */ + public final Counter failedStoreDeletionAttempts; + public LocalStoreMonitorMetrics(String prefix, MetricsRegistry registry) { super(prefix, registry); diskSpaceFreedInBytes = newCounter("diskSpaceFreedInBytes"); noOfDeletedTaskPartitionStores = newCounter("noOfDeletedTaskPartitionStores"); +failedStoreDeletionAttempts = newCounter("failedStoreDeletionAttempts"); } }
[jira] [Commented] (SAMZA-1492) Add exception counter to LocalStoreMonitor.
[ https://issues.apache.org/jira/browse/SAMZA-1492?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16251604#comment-16251604 ] ASF GitHub Bot commented on SAMZA-1492: --- Github user asfgit closed the pull request at: https://github.com/apache/samza/pull/357 > Add exception counter to LocalStoreMonitor. > --- > > Key: SAMZA-1492 > URL: https://issues.apache.org/jira/browse/SAMZA-1492 > Project: Samza > Issue Type: Bug >Reporter: Shanthoosh Venkataraman >Assignee: Shanthoosh Venkataraman > Fix For: 0.14.0 > > > Add storeGCExceptionCounter to LocalStoreMonitor(as a part of > LocalStoreMonitorMetrics) to enable alerts setup in an production environment. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Resolved] (SAMZA-1492) Add exception counter to LocalStoreMonitor.
[ https://issues.apache.org/jira/browse/SAMZA-1492?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jake Maes resolved SAMZA-1492. -- Resolution: Fixed Fix Version/s: 0.14.0 Issue resolved by pull request 357 [https://github.com/apache/samza/pull/357] > Add exception counter to LocalStoreMonitor. > --- > > Key: SAMZA-1492 > URL: https://issues.apache.org/jira/browse/SAMZA-1492 > Project: Samza > Issue Type: Bug >Reporter: Shanthoosh Venkataraman >Assignee: Shanthoosh Venkataraman > Fix For: 0.14.0 > > > Add storeGCExceptionCounter to LocalStoreMonitor(as a part of > LocalStoreMonitorMetrics) to enable alerts setup in an production environment. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1493) Samza Standalone should create changelog topics
Prateek Maheshwari created SAMZA-1493: - Summary: Samza Standalone should create changelog topics Key: SAMZA-1493 URL: https://issues.apache.org/jira/browse/SAMZA-1493 Project: Samza Issue Type: Bug Reporter: Prateek Maheshwari Assignee: Shanthoosh Venkataraman Fix For: 0.14.0 Currently Samza standalone doesn't create changelog topics through the JobModelManager. This means that they get autocreated with the default number of partitions when they're read/written to. Needs to be fixed for supporting durable state for high level API. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1494) Flush operator state on end of stream
Prateek Maheshwari created SAMZA-1494: - Summary: Flush operator state on end of stream Key: SAMZA-1494 URL: https://issues.apache.org/jira/browse/SAMZA-1494 Project: Samza Issue Type: Bug Reporter: Prateek Maheshwari Assignee: Jagadish Fix For: 0.14.0 Stateful operators should flush all buffered state when end of stream is reached. Otherwise the container may shut down with open windows and lead to message loss. As of 0.14, only window operator needs to do this. When we add outer/left/right joins they'll need to do it as well. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1499) Make operator storage engine factory configurable
Prateek Maheshwari created SAMZA-1499: - Summary: Make operator storage engine factory configurable Key: SAMZA-1499 URL: https://issues.apache.org/jira/browse/SAMZA-1499 Project: Samza Issue Type: Task Reporter: Prateek Maheshwari Assignee: Prateek Maheshwari Fix For: 0.14.0 Currently stateful operators require a dependency on RocksDB for maintaining local state. We should allow the storage engine factory to be configurable. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
samza git commit: SAMZA-1490: Fix TestRepartitionJoinWindowApp
Repository: samza Updated Branches: refs/heads/master 26dd9c14c -> 42c194040 SAMZA-1490: Fix TestRepartitionJoinWindowApp Author: Dong LinReviewers: Prateek Maheshwari Closes #356 from lindong28/SAMZA-1490 Project: http://git-wip-us.apache.org/repos/asf/samza/repo Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/42c19404 Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/42c19404 Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/42c19404 Branch: refs/heads/master Commit: 42c194040c9adea29eddc92dc58ca01a59a22445 Parents: 26dd9c1 Author: Dong Lin Authored: Tue Nov 14 10:48:50 2017 -0800 Committer: Prateek Maheshwari Committed: Tue Nov 14 10:48:50 2017 -0800 -- .../test/operator/TestRepartitionJoinWindowApp.java| 13 +++-- 1 file changed, 3 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/samza/blob/42c19404/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java -- diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java index 117f97b..49611bb 100644 --- a/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java +++ b/samza-test/src/test/java/org/apache/samza/test/operator/TestRepartitionJoinWindowApp.java @@ -44,19 +44,16 @@ public class TestRepartitionJoinWindowApp extends StreamApplicationIntegrationTe // create events for the following user activity. // userId: (viewId, pageId, (adIds)) -// u1: (v1, p1, (a1, a2)), (v2, p2, (a3)) -// u2: (v3, p1, (a1, a2, a4)), (v4, p3, (a5)) +// u1: (v1, p1, (a1)), (v2, p2, (a3)) +// u2: (v3, p1, (a1)), (v4, p3, (a5)) produceMessage(PAGE_VIEWS, 0, "p1", "{\"viewId\":\"v1\",\"pageId\":\"p1\",\"userId\":\"u1\"}"); produceMessage(PAGE_VIEWS, 1, "p2", "{\"viewId\":\"v2\",\"pageId\":\"p2\",\"userId\":\"u1\"}"); produceMessage(PAGE_VIEWS, 0, "p1", "{\"viewId\":\"v3\",\"pageId\":\"p1\",\"userId\":\"u2\"}"); produceMessage(PAGE_VIEWS, 1, "p3", "{\"viewId\":\"v4\",\"pageId\":\"p3\",\"userId\":\"u2\"}"); produceMessage(AD_CLICKS, 0, "a1", "{\"viewId\":\"v1\",\"adId\":\"a1\"}"); -produceMessage(AD_CLICKS, 1, "a2", "{\"viewId\":\"v1\",\"adId\":\"a2\"}"); produceMessage(AD_CLICKS, 0, "a3", "{\"viewId\":\"v2\",\"adId\":\"a3\"}"); produceMessage(AD_CLICKS, 0, "a1", "{\"viewId\":\"v3\",\"adId\":\"a1\"}"); -produceMessage(AD_CLICKS, 1, "a2", "{\"viewId\":\"v3\",\"adId\":\"a2\"}"); -produceMessage(AD_CLICKS, 1, "a4", "{\"viewId\":\"v3\",\"adId\":\"a4\"}"); produceMessage(AD_CLICKS, 0, "a5", "{\"viewId\":\"v4\",\"adId\":\"a5\"}"); // run the application @@ -71,11 +68,7 @@ public class TestRepartitionJoinWindowApp extends StreamApplicationIntegrationTe String key = message.key(); String value = message.value(); Assert.assertTrue(key.equals("u1") || key.equals("u2")); - if ("u1".equals(key)) { -Assert.assertEquals("3", value); - } else { -Assert.assertEquals("4", value); - } + Assert.assertEquals("2", value); } } }
[jira] [Commented] (SAMZA-1487) Disable Flaky Zk Integration tests.
[ https://issues.apache.org/jira/browse/SAMZA-1487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16251955#comment-16251955 ] ASF GitHub Bot commented on SAMZA-1487: --- Github user asfgit closed the pull request at: https://github.com/apache/samza/pull/355 > Disable Flaky Zk Integration tests. > --- > > Key: SAMZA-1487 > URL: https://issues.apache.org/jira/browse/SAMZA-1487 > Project: Samza > Issue Type: Bug >Reporter: Shanthoosh Venkataraman >Assignee: Shanthoosh Venkataraman > > These tests will be fixed and enabled back via JIRA's: SAMZA-1449, > SAMZA-1450, SAMZA-1488 -- This message was sent by Atlassian JIRA (v6.4.14#64029)
samza git commit: SAMZA-1487: Disable Flaky Zk Integration tests.
Repository: samza Updated Branches: refs/heads/master 42c194040 -> 20fe6ec61 SAMZA-1487: Disable Flaky Zk Integration tests. Author: Shanthoosh VenkataramanReviewers: Yi Pan Closes #355 from shanthoosh/master Project: http://git-wip-us.apache.org/repos/asf/samza/repo Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/20fe6ec6 Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/20fe6ec6 Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/20fe6ec6 Branch: refs/heads/master Commit: 20fe6ec6188a21c670b12bbfd1bc8f2f031a7b12 Parents: 42c1940 Author: Shanthoosh Venkataraman Authored: Tue Nov 14 10:54:32 2017 -0800 Committer: Prateek Maheshwari Committed: Tue Nov 14 10:54:32 2017 -0800 -- .../zk/TestZkBarrierForVersionUpgrade.java | 23 ++-- 1 file changed, 11 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/samza/blob/20fe6ec6/samza-core/src/test/java/org/apache/samza/zk/TestZkBarrierForVersionUpgrade.java -- diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkBarrierForVersionUpgrade.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkBarrierForVersionUpgrade.java index 3dd1bd5..7689901 100644 --- a/samza-core/src/test/java/org/apache/samza/zk/TestZkBarrierForVersionUpgrade.java +++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkBarrierForVersionUpgrade.java @@ -23,18 +23,17 @@ import org.I0Itec.zkclient.ZkClient; import org.apache.samza.config.ZkConfig; import org.apache.samza.testUtils.EmbeddedZookeeper; import org.apache.samza.util.NoOpMetricsRegistry; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; - +//import org.junit.After; +//import org.junit.AfterClass; +//import org.junit.Before; +//import org.junit.BeforeClass; +//import org.junit.Test; // TODO: Rename this such that it is clear that it is an integration test and NOT unit test public class TestZkBarrierForVersionUpgrade { @@ -43,14 +42,14 @@ public class TestZkBarrierForVersionUpgrade { private ZkUtils zkUtils; private ZkUtils zkUtils1; - @BeforeClass + //@BeforeClass public static void test() { zkServer = new EmbeddedZookeeper(); zkServer.setup(); testZkConnectionString = "127.0.0.1:" + zkServer.getPort(); } - @Before + //@Before public void testSetup() { ZkClient zkClient = new ZkClient(testZkConnectionString, ZkConfig.DEFAULT_SESSION_TIMEOUT_MS, ZkConfig.DEFAULT_CONNECTION_TIMEOUT_MS); this.zkUtils = new ZkUtils(new ZkKeyBuilder("group1"), zkClient, ZkConfig.DEFAULT_CONNECTION_TIMEOUT_MS, new NoOpMetricsRegistry()); @@ -58,18 +57,18 @@ public class TestZkBarrierForVersionUpgrade { this.zkUtils1 = new ZkUtils(new ZkKeyBuilder("group1"), zkClient1, ZkConfig.DEFAULT_CONNECTION_TIMEOUT_MS, new NoOpMetricsRegistry()); } - @After + //@After public void testTearDown() { zkUtils.close(); zkUtils1.close(); } - @AfterClass + //@AfterClass public static void teardown() { zkServer.teardown(); } - @Test + //@Test public void testZkBarrierForVersionUpgrade() { String barrierId = zkUtils.getKeyBuilder().getRootPath() + "/b1"; String ver = "1"; @@ -140,7 +139,7 @@ public class TestZkBarrierForVersionUpgrade { Assert.assertEquals(2, stateChangedCalled.get()); } - @Test + //@Test public void testZkBarrierForVersionUpgradeWithTimeOut() { String barrierId = zkUtils1.getKeyBuilder().getRootPath() + "/barrierTimeout"; String ver = "1";
samza git commit: SAMZA-1474: Bump up rocksdb version to 5.7.3 to include licensing changes
Repository: samza Updated Branches: refs/heads/master 20fe6ec61 -> 3750f5e24 SAMZA-1474: Bump up rocksdb version to 5.7.3 to include licensing changes You can find more details on the bug fixes and API changes [here](https://github.com/facebook/rocksdb/releases). I upgraded to 5.7.3 since 5.8.0 has a regression [KAFKA-6100](https://issues.apache.org/jira/browse/KAFKA-6100) All of our tests passed locally. I will monitor travis for failures. Author: Bharath KumarasubramanianReviewers: Boris Shkolnik Closes #344 from bharathkk/master Project: http://git-wip-us.apache.org/repos/asf/samza/repo Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/3750f5e2 Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/3750f5e2 Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/3750f5e2 Branch: refs/heads/master Commit: 3750f5e24b601837b29f3654189159e7f8b828c1 Parents: 20fe6ec Author: Bharath Kumarasubramanian Authored: Tue Nov 14 11:00:34 2017 -0800 Committer: Prateek Maheshwari Committed: Tue Nov 14 11:00:34 2017 -0800 -- NOTICE| 2 -- gradle/dependency-versions.gradle | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/samza/blob/3750f5e2/NOTICE -- diff --git a/NOTICE b/NOTICE index 8024af7..3352dda 100644 --- a/NOTICE +++ b/NOTICE @@ -3,5 +3,3 @@ Copyright 2014 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). - -This product uses RocksDB which carries Facebook BSD+patents license. http://git-wip-us.apache.org/repos/asf/samza/blob/3750f5e2/gradle/dependency-versions.gradle -- diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index cedc979..a8af9a8 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -35,7 +35,7 @@ metricsVersion = "2.2.0" mockitoVersion = "1.10.19" powerMockVersion = "1.6.6" - rocksdbVersion = "5.0.1" + rocksdbVersion = "5.7.3" scalaTestVersion = "3.0.1" slf4jVersion = "1.6.2" yarnVersion = "2.6.1"
[jira] [Commented] (SAMZA-1474) Upgrade RocksDB to include apache licensing changes
[ https://issues.apache.org/jira/browse/SAMZA-1474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16251966#comment-16251966 ] ASF GitHub Bot commented on SAMZA-1474: --- Github user asfgit closed the pull request at: https://github.com/apache/samza/pull/344 > Upgrade RocksDB to include apache licensing changes > --- > > Key: SAMZA-1474 > URL: https://issues.apache.org/jira/browse/SAMZA-1474 > Project: Samza > Issue Type: Task >Reporter: Bharath Kumarasubramanian >Assignee: Bharath Kumarasubramanian > Fix For: 0.14.0 > > > RocksDB previously used BSD+patens license and had problems with patent grant > wording. > They have updated their license to Apache + GPL v2. Apache requires that all > release of Samza after Aug 31st needs to upgrade to 5.5.4 or higher. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1495) Mark intermediate streams as higher priority than input streams in planner
Prateek Maheshwari created SAMZA-1495: - Summary: Mark intermediate streams as higher priority than input streams in planner Key: SAMZA-1495 URL: https://issues.apache.org/jira/browse/SAMZA-1495 Project: Samza Issue Type: Bug Reporter: Prateek Maheshwari Assignee: Prateek Maheshwari Fix For: 0.14.0 Intermediate streams should be set as higher priority than input streams in the High Level API to prevent data from getting buffered and delayed. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1496) Update Samza's ApplicationMaster to work with Yarn's host affinity and node labels
Prateek Maheshwari created SAMZA-1496: - Summary: Update Samza's ApplicationMaster to work with Yarn's host affinity and node labels Key: SAMZA-1496 URL: https://issues.apache.org/jira/browse/SAMZA-1496 Project: Samza Issue Type: Bug Reporter: Prateek Maheshwari Assignee: Prateek Maheshwari Fix For: 0.14.0 Samza's current version of Yarn doesn't support making resource requests with both host affinity and node labels. We need a workaround to move to capacity scheduler with node labels for resource reservation. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (SAMZA-1498) Add an onTimer method to operator functions
Prateek Maheshwari created SAMZA-1498: - Summary: Add an onTimer method to operator functions Key: SAMZA-1498 URL: https://issues.apache.org/jira/browse/SAMZA-1498 Project: Samza Issue Type: Bug Reporter: Prateek Maheshwari Assignee: Prateek Maheshwari Priority: Minor Fix For: 0.14.0 Expose an 'onTimer/window' method in operator functions to support periodic callbacks. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (SAMZA-1490) Fix TestRepartitionJoinWindowApp
[ https://issues.apache.org/jira/browse/SAMZA-1490?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16251935#comment-16251935 ] ASF GitHub Bot commented on SAMZA-1490: --- Github user asfgit closed the pull request at: https://github.com/apache/samza/pull/356 > Fix TestRepartitionJoinWindowApp > > > Key: SAMZA-1490 > URL: https://issues.apache.org/jira/browse/SAMZA-1490 > Project: Samza > Issue Type: Bug > Environment: Currently TestRepartitionJoinWindowApp will fail > randomly because the output from join operator may vary depending on the > order of message delivery from the input streams. We should fix this test so > that it does not fail randomly. >Reporter: Dong Lin >Assignee: Dong Lin > -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Resolved] (SAMZA-1490) Fix TestRepartitionJoinWindowApp
[ https://issues.apache.org/jira/browse/SAMZA-1490?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Prateek Maheshwari resolved SAMZA-1490. --- Resolution: Fixed Thanks for the patch! > Fix TestRepartitionJoinWindowApp > > > Key: SAMZA-1490 > URL: https://issues.apache.org/jira/browse/SAMZA-1490 > Project: Samza > Issue Type: Bug > Environment: Currently TestRepartitionJoinWindowApp will fail > randomly because the output from join operator may vary depending on the > order of message delivery from the input streams. We should fix this test so > that it does not fail randomly. >Reporter: Dong Lin >Assignee: Dong Lin > -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Resolved] (SAMZA-1474) Upgrade RocksDB to include apache licensing changes
[ https://issues.apache.org/jira/browse/SAMZA-1474?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Prateek Maheshwari resolved SAMZA-1474. --- Resolution: Fixed > Upgrade RocksDB to include apache licensing changes > --- > > Key: SAMZA-1474 > URL: https://issues.apache.org/jira/browse/SAMZA-1474 > Project: Samza > Issue Type: Task >Reporter: Bharath Kumarasubramanian >Assignee: Bharath Kumarasubramanian > Fix For: 0.14.0 > > > RocksDB previously used BSD+patens license and had problems with patent grant > wording. > They have updated their license to Apache + GPL v2. Apache requires that all > release of Samza after Aug 31st needs to upgrade to 5.5.4 or higher. -- This message was sent by Atlassian JIRA (v6.4.14#64029)