[jira] [Updated] (RATIS-135) Closed segments contribute to leaked memory as eviction policies are not honoured
[ https://issues.apache.org/jira/browse/RATIS-135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mukul Kumar Singh updated RATIS-135: Attachment: RATIS-135.001.patch > Closed segments contribute to leaked memory as eviction policies are not > honoured > - > > Key: RATIS-135 > URL: https://issues.apache.org/jira/browse/RATIS-135 > Project: Ratis > Issue Type: Bug >Reporter: Mukul Kumar Singh >Assignee: Mukul Kumar Singh > Attachments: RATIS-135.001.patch > > > Closed segments leak memory as eviction does not remove these segments from > closed segments list. > {code} > void evictCache(long[] followerIndices, long flushedIndex, > long lastAppliedIndex) { > List toEvict = evictionPolicy.evict(followerIndices, > flushedIndex, lastAppliedIndex, closedSegments, maxCachedSegments); > for (LogSegment s : toEvict) { > s.evictCache(); > } > {code} > After the segments have been marked for eviction, they should be removed from > queue as well. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (RATIS-135) Closed segments contribute to leaked memory as eviction policies are not honoured
Mukul Kumar Singh created RATIS-135: --- Summary: Closed segments contribute to leaked memory as eviction policies are not honoured Key: RATIS-135 URL: https://issues.apache.org/jira/browse/RATIS-135 Project: Ratis Issue Type: Bug Reporter: Mukul Kumar Singh Assignee: Mukul Kumar Singh Closed segments leak memory as eviction does not remove these segments from closed segments list. {code} void evictCache(long[] followerIndices, long flushedIndex, long lastAppliedIndex) { List toEvict = evictionPolicy.evict(followerIndices, flushedIndex, lastAppliedIndex, closedSegments, maxCachedSegments); for (LogSegment s : toEvict) { s.evictCache(); } {code} After the segments have been marked for eviction, they should be removed from queue as well. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (RATIS-72) Enable Ratis to run on high-performance RDMA networks
[ https://issues.apache.org/jira/browse/RATIS-72?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16239858#comment-16239858 ] Jitendra Nath Pandey commented on RATIS-72: --- [~atrivedi], Could you please run TestRaftWithSimulatedRpc individually? {code} mvn test -Dtest=TestRaftWithSimulatedRpc {code} I tried a few runs, and it passes for me on the latest code. Does it fail consistently for you? There have been some maven changes recently. You would need to do {{mvn package -DskipTests}}, after {{mvn clean}} to get shaded jars correctly built. See latest BUILDING.md file. RATIS-132, once fixed, will get {{mvn clean install}} working. > Enable Ratis to run on high-performance RDMA networks > - > > Key: RATIS-72 > URL: https://issues.apache.org/jira/browse/RATIS-72 > Project: Ratis > Issue Type: New Feature >Reporter: Animesh Trivedi >Assignee: Animesh Trivedi >Priority: Minor > Attachments: failed-test > > > We want to enable Ratis to run on high-performance RDMA networks. There has > already been quite a bit of activity in leveraging RDMA networks in modern > data processing stacks. Distributed consensus is one of the problems that can > be accelerated using high-performance/RDMA messaging. > We start by implementing a Ratis RPC implementation using DiSNI and DaPRC > modules (https://github.com/zrlio/) for RDMA networks. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Comment Edited] (RATIS-72) Enable Ratis to run on high-performance RDMA networks
[ https://issues.apache.org/jira/browse/RATIS-72?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16239699#comment-16239699 ] Animesh Trivedi edited comment on RATIS-72 at 11/5/17 7:35 PM: --- Hi [~jnp], many thanks for offering your help. I am trying to finish this implementation now. One thing that would be helpful is to get the maven unit tests working. Currently, I when I run mvn test, I get : [...] Failed tests: TestRaftWithSimulatedRpc>RaftBasicTests.testBasicAppendEntries:127->RaftBasicTests.lambda$testBasicAppendEntries$1:127 expected:<10> but was:<11> Tests run: 65, Failures: 1, Errors: 0, Skipped: 0 [INFO] [INFO] Reactor Summary: [INFO] [INFO] Apache Ratis ... SUCCESS [ 1.014 s] [INFO] Apache Ratis Proto Shaded .. SUCCESS [ 3.353 s] [INFO] Apache Ratis Common SUCCESS [ 5.008 s] [INFO] Apache Ratis Client SUCCESS [ 0.080 s] [INFO] Apache Ratis Hadoop Shaded . SUCCESS [ 1.397 s] [INFO] Apache Ratis Server FAILURE [01:41 min] [INFO] Apache Ratis Hadoop Support SKIPPED [INFO] Apache Ratis gRPC Support .. SKIPPED [INFO] Apache Ratis Netty Support . SKIPPED [INFO] Apache Ratis Replicated Map SKIPPED [INFO] Apache Ratis Project Assembly .. SKIPPED [INFO] Apache Ratis Examples .. SKIPPED [INFO] [INFO] BUILD FAILURE [INFO] As I am trying get the similar functionality implemented as the gRPC or netty code, it would be helpful to have the basic tests working. Naturally, debugging in a distributed setting requires another level of testing. Thanks a lot ! was (Author: atrivedi): Hi [~jnp]], many thanks for offering your help. I am trying to finish this implementation now. One thing that would be helpful is to get the maven unit tests working. Currently, I when I run mvn test, I get : [...] Failed tests: TestRaftWithSimulatedRpc>RaftBasicTests.testBasicAppendEntries:127->RaftBasicTests.lambda$testBasicAppendEntries$1:127 expected:<10> but was:<11> Tests run: 65, Failures: 1, Errors: 0, Skipped: 0 [INFO] [INFO] Reactor Summary: [INFO] [INFO] Apache Ratis ... SUCCESS [ 1.014 s] [INFO] Apache Ratis Proto Shaded .. SUCCESS [ 3.353 s] [INFO] Apache Ratis Common SUCCESS [ 5.008 s] [INFO] Apache Ratis Client SUCCESS [ 0.080 s] [INFO] Apache Ratis Hadoop Shaded . SUCCESS [ 1.397 s] [INFO] Apache Ratis Server FAILURE [01:41 min] [INFO] Apache Ratis Hadoop Support SKIPPED [INFO] Apache Ratis gRPC Support .. SKIPPED [INFO] Apache Ratis Netty Support . SKIPPED [INFO] Apache Ratis Replicated Map SKIPPED [INFO] Apache Ratis Project Assembly .. SKIPPED [INFO] Apache Ratis Examples .. SKIPPED [INFO] [INFO] BUILD FAILURE [INFO] As I am trying get the similar functionality implemented as the gRPC or netty code, it would be helpful to have the basic tests working. Naturally, debugging in a distributed setting requires another level of testing. Thanks a lot ! > Enable Ratis to run on high-performance RDMA networks > - > > Key: RATIS-72 > URL: https://issues.apache.org/jira/browse/RATIS-72 > Project: Ratis > Issue Type: New Feature >Reporter: Animesh Trivedi >Assignee: Animesh Trivedi >Priority: Minor > Attachments: failed-test > > > We want to enable Ratis to run on high-performance RDMA networks. There has > already been quite a bit of activity in leveraging RDMA networks in modern > data processing stacks. Distributed consensus is one of the problems that can > be accelerated using high-performance/RDMA messaging. > We start by implementing a Ratis RPC implementation using DiSNI and DaPRC > modules (https://github.com/zrlio/) for RDMA networks. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (RATIS-72) Enable Ratis to run on high-performance RDMA networks
[ https://issues.apache.org/jira/browse/RATIS-72?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16239699#comment-16239699 ] Animesh Trivedi commented on RATIS-72: -- Hi [~jnp]], many thanks for offering your help. I am trying to finish this implementation now. One thing that would be helpful is to get the maven unit tests working. Currently, I when I run mvn test, I get : [...] Failed tests: TestRaftWithSimulatedRpc>RaftBasicTests.testBasicAppendEntries:127->RaftBasicTests.lambda$testBasicAppendEntries$1:127 expected:<10> but was:<11> Tests run: 65, Failures: 1, Errors: 0, Skipped: 0 [INFO] [INFO] Reactor Summary: [INFO] [INFO] Apache Ratis ... SUCCESS [ 1.014 s] [INFO] Apache Ratis Proto Shaded .. SUCCESS [ 3.353 s] [INFO] Apache Ratis Common SUCCESS [ 5.008 s] [INFO] Apache Ratis Client SUCCESS [ 0.080 s] [INFO] Apache Ratis Hadoop Shaded . SUCCESS [ 1.397 s] [INFO] Apache Ratis Server FAILURE [01:41 min] [INFO] Apache Ratis Hadoop Support SKIPPED [INFO] Apache Ratis gRPC Support .. SKIPPED [INFO] Apache Ratis Netty Support . SKIPPED [INFO] Apache Ratis Replicated Map SKIPPED [INFO] Apache Ratis Project Assembly .. SKIPPED [INFO] Apache Ratis Examples .. SKIPPED [INFO] [INFO] BUILD FAILURE [INFO] As I am trying get the similar functionality implemented as the gRPC or netty code, it would be helpful to have the basic tests working. Naturally, debugging in a distributed setting requires another level of testing. Thanks a lot ! > Enable Ratis to run on high-performance RDMA networks > - > > Key: RATIS-72 > URL: https://issues.apache.org/jira/browse/RATIS-72 > Project: Ratis > Issue Type: New Feature >Reporter: Animesh Trivedi >Assignee: Animesh Trivedi >Priority: Minor > Attachments: failed-test > > > We want to enable Ratis to run on high-performance RDMA networks. There has > already been quite a bit of activity in leveraging RDMA networks in modern > data processing stacks. Distributed consensus is one of the problems that can > be accelerated using high-performance/RDMA messaging. > We start by implementing a Ratis RPC implementation using DiSNI and DaPRC > modules (https://github.com/zrlio/) for RDMA networks. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (RATIS-134) PeerProxyMap#addPeers should add peers only if it is not already present in the proxymap
[ https://issues.apache.org/jira/browse/RATIS-134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16239427#comment-16239427 ] Hadoop QA commented on RATIS-134: - | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 1m 31s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:blue}0{color} | {color:blue} findbugs {color} | {color:blue} 0m 0s{color} | {color:blue} Findbugs executables are not available. {color} | | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:red}-1{color} | {color:red} test4tests {color} | {color:red} 0m 0s{color} | {color:red} The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. {color} | || || || || {color:brown} master Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 0m 19s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 8s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 10s{color} | {color:green} master passed {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 11s{color} | {color:green} master passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 0m 8s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 8s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 8s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 10s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 10s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:green}+1{color} | {color:green} unit {color} | {color:green} 0m 14s{color} | {color:green} ratis-common in the patch passed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 6s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black} 3m 30s{color} | {color:black} {color} | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/ratis:date2017-11-05 | | JIRA Issue | RATIS-134 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12896115/RATIS-134.001.patch | | Optional Tests | asflicense javac javadoc unit findbugs checkstyle compile | | uname | Linux 34d50a531c80 3.13.0-123-generic #172-Ubuntu SMP Mon Jun 26 18:04:35 UTC 2017 x86_64 x86_64 x86_64 GNU/Linux | | Build tool | maven | | Personality | /home/jenkins/jenkins-slave/workspace/PreCommit-RATIS-Build/yetus-personality.sh | | git revision | master / 6d0d114 | | Default Java | 1.8.0_131 | | Test Results | https://builds.apache.org/job/PreCommit-RATIS-Build/15/testReport/ | | modules | C: ratis-common U: ratis-common | | Console output | https://builds.apache.org/job/PreCommit-RATIS-Build/15/console | | Powered by | Apache Yetus 0.5.0 http://yetus.apache.org | This message was automatically generated. > PeerProxyMap#addPeers should add peers only if it is not already present in > the proxymap > > > Key: RATIS-134 > URL: https://issues.apache.org/jira/browse/RATIS-134 > Project: Ratis > Issue Type: Bug >Reporter: Mukul Kumar Singh >Assignee: Mukul Kumar Singh > Attachments: RATIS-134.001.patch > > > {{PeerProxyMap#addPeers}} currently adds peers to the proxyMap without > checkint the current state. This should be optimized to allow addition only > if the peer is not present in the proxyMap. > {code} > public void addPeers(Iterable newPeers) { > for(RaftPeer p : newPeers) { > peers.put(p.getId(), new PeerAndProxy(p)); > } > } > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Updated] (RATIS-134) PeerProxyMap#addPeers should add peers only if it is not already present in the proxymap
[ https://issues.apache.org/jira/browse/RATIS-134?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mukul Kumar Singh updated RATIS-134: Attachment: RATIS-134.001.patch > PeerProxyMap#addPeers should add peers only if it is not already present in > the proxymap > > > Key: RATIS-134 > URL: https://issues.apache.org/jira/browse/RATIS-134 > Project: Ratis > Issue Type: Bug >Reporter: Mukul Kumar Singh >Assignee: Mukul Kumar Singh > Attachments: RATIS-134.001.patch > > > {{PeerProxyMap#addPeers}} currently adds peers to the proxyMap without > checkint the current state. This should be optimized to allow addition only > if the peer is not present in the proxyMap. > {code} > public void addPeers(Iterable newPeers) { > for(RaftPeer p : newPeers) { > peers.put(p.getId(), new PeerAndProxy(p)); > } > } > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Created] (RATIS-134) PeerProxyMap#addPeers should add peers only if it is not already present in the proxymap
Mukul Kumar Singh created RATIS-134: --- Summary: PeerProxyMap#addPeers should add peers only if it is not already present in the proxymap Key: RATIS-134 URL: https://issues.apache.org/jira/browse/RATIS-134 Project: Ratis Issue Type: Bug Reporter: Mukul Kumar Singh Assignee: Mukul Kumar Singh {{PeerProxyMap#addPeers}} currently adds peers to the proxyMap without checkint the current state. This should be optimized to allow addition only if the peer is not present in the proxyMap. {code} public void addPeers(Iterable newPeers) { for(RaftPeer p : newPeers) { peers.put(p.getId(), new PeerAndProxy(p)); } } {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)