You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by enixon <gi...@git.apache.org> on 2017/03/15 22:36:12 UTC

[GitHub] zookeeper pull request #195: Promote local session to global when ephemeral ...

GitHub user enixon opened a pull request:

    https://github.com/apache/zookeeper/pull/195

    Promote local session to global when ephemeral created in multi-op

    Patch and unit test for issue.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/enixon/zookeeper ZOOKEEPER-2725

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/zookeeper/pull/195.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 #195
    
----
commit d9050bece1f4bc5738dd5f4edd256f5df0797d49
Author: Brian Nixon <ni...@fb.com>
Date:   2017-03-09T02:48:19Z

    Promote local session to global when ephemeral created in multi-op

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r106797306
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java ---
    @@ -62,18 +64,40 @@ public Request checkUpgradeSession(Request request)
             // This is called by the request processor thread (either follower
             // or observer request processor), which is unique to a learner.
             // So will not be called concurrently by two threads.
    -        if (request.type != OpCode.create ||
    +        if ((request.type != OpCode.create && request.type != OpCode.create2 && request.type != OpCode.multi) ||
                 !upgradeableSessionTracker.isLocalSession(request.sessionId)) {
                 return null;
             }
    -        CreateRequest createRequest = new CreateRequest();
    -        request.request.rewind();
    -        ByteBufferInputStream.byteBuffer2Record(request.request, createRequest);
    -        request.request.rewind();
    -        CreateMode createMode = CreateMode.fromFlag(createRequest.getFlags());
    -        if (!createMode.isEphemeral()) {
    -            return null;
    +
    +        if (OpCode.multi == request.type) {
    +            MultiTransactionRecord multiTransactionRecord = new MultiTransactionRecord();
    +            request.request.rewind();
    +            ByteBufferInputStream.byteBuffer2Record(request.request, multiTransactionRecord);
    +            request.request.rewind();
    +            boolean containsEphemeralCreate = false;
    +            for (Op op : multiTransactionRecord) {
    +                if (op.getType() == OpCode.create || op.getType() == OpCode.create2) {
    +                    CreateRequest createRequest = (CreateRequest)op.toRequestRecord();
    +                    CreateMode createMode = CreateMode.fromFlag(createRequest.getFlags());
    +                    if (createMode.isEphemeral()) {
    +                        containsEphemeralCreate = true;
    --- End diff --
    
    I think we can just return null here (or break the loop) - there is no need to check rest of ops because as long as there is an ephemeral / create we'd upgrade session?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #195: ZOOKEEPER-2725: Promote local session to global when e...

Posted by enixon <gi...@git.apache.org>.
Github user enixon commented on the issue:

    https://github.com/apache/zookeeper/pull/195
  
    Proposed changes sound good, addressed with additional commit


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/zookeeper/pull/195


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r106797356
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,87 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +        waitForClient(zk, ZooKeeper.States.CONNECTED);
    +
    +        String data = "test";
    +        String path = "/ephemeralcreatemultiop";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        List<OpResult> multi = null;
    +        LOG.info("RUNNING MULTI-OP");
    +        try {
    +            multi = zk.multi(Arrays.asList(
    +                    Op.setData(path, data.getBytes(), 0),
    +                    Op.create(path + "/e", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL),
    +                    Op.create(path + "/p", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
    +                    Op.create(path + "/q", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
    +            ));
    +        } catch (KeeperException.SessionExpiredException e) {
    +            LOG.info("SESSION EXPIRED", e);
    --- End diff --
    
    We are not expected to get a SessionExpiredException here right with this patch, right? So might worth to put an assert here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r106797340
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,87 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +        waitForClient(zk, ZooKeeper.States.CONNECTED);
    --- End diff --
    
    Right, there is no need for this check (which use polling which is not recommended.). createClient already ensures that zk is in connected state via watchers. The only benefit of additional check here might be that in case of flaky test environment, a zk might loss connection to server right after createClient, but in that case we might want to wrap the subsequent test code (i.e. create) using retry with timeout logic.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r107051228
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,135 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.proto.CreateRequest;
    +import org.apache.zookeeper.proto.GetDataRequest;
    +import org.apache.zookeeper.server.quorum.QuorumPeer;
    +import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer;
    +import org.apache.zookeeper.server.quorum.UpgradeableSessionTracker;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +
    +        String data = "test";
    +        String path = "/ephemeralcreatemultiop";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        QuorumZooKeeperServer server = getConnectedServer(zk.getSessionId());
    +        Assert.assertNotNull("unable to find server interlocutor", server);
    +        UpgradeableSessionTracker sessionTracker = (UpgradeableSessionTracker)server.getSessionTracker();
    +        Assert.assertFalse("session already global", sessionTracker.isGlobalSession(zk.getSessionId()));
    +
    +        List<OpResult> multi = null;
    +        try {
    +            multi = zk.multi(Arrays.asList(
    +                    Op.setData(path, data.getBytes(), 0),
    +                    Op.create(path + "/e", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL),
    +                    Op.create(path + "/p", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
    +                    Op.create(path + "/q", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
    +            ));
    +        } catch (KeeperException.SessionExpiredException e) {
    +            // the scenario that inspired this unit test
    +            Assert.fail("received session expired for a session promotion in a multi-op");
    +        }
    +
    +        Assert.assertNotNull(multi);
    +        Assert.assertEquals(4, multi.size());
    +        Assert.assertEquals(data, new String(zk.getData(path + "/e", false, null)));
    +        Assert.assertEquals(data, new String(zk.getData(path + "/p", false, null)));
    +        Assert.assertEquals(data, new String(zk.getData(path + "/q", false, null)));
    +        Assert.assertTrue("session not promoted", sessionTracker.isGlobalSession(zk.getSessionId()));
    +    }
    +
    +    @Test
    +    public void directCheckUpgradeSessionTest() throws IOException, InterruptedException, KeeperException {
    +        final ZooKeeper zk = createClient();
    +
    +        String path = "/directcheckupgradesession";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        QuorumZooKeeperServer server = getConnectedServer(zk.getSessionId());
    +        Assert.assertNotNull("unable to find server interlocutor", server);
    +
    +        Request readRequest = makeGetDataRequest(path, zk.getSessionId());
    +        Request createRequest = makeCreateRequest(path + "/e", zk.getSessionId());
    +        Assert.assertNull("tried to upgrade on a read", server.checkUpgradeSession(readRequest));
    +        Assert.assertNotNull("failed to upgrade on a create", server.checkUpgradeSession(createRequest));
    +        Assert.assertNull("tried to upgrade after successful promotion",
    +                server.checkUpgradeSession(createRequest));
    +    }
    +
    +    private Request makeGetDataRequest(String path, long sessionId) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    +        BinaryOutputArchive boa = BinaryOutputArchive.getArchive(boas);
    +        GetDataRequest getDataRequest = new GetDataRequest(path, false);
    +        getDataRequest.serialize(boa, "request");
    +        ByteBuffer bb = ByteBuffer.wrap(boas.toByteArray());
    +        return new Request(null, sessionId, 1, ZooDefs.OpCode.getData, bb, new ArrayList<>());
    +    }
    +
    +    private Request makeCreateRequest(String path, long sessionId) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    +        BinaryOutputArchive boa = BinaryOutputArchive.getArchive(boas);
    +        CreateRequest createRequest = new CreateRequest(path,
    +                "data".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL.toFlag());
    +        createRequest.serialize(boa, "request");
    +        ByteBuffer bb = ByteBuffer.wrap(boas.toByteArray());
    +        return new Request(null, sessionId, 1, ZooDefs.OpCode.create2, bb, new ArrayList<>());
    --- End diff --
    
    Same here (`new ArrayList<>`)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r107051170
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,135 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.proto.CreateRequest;
    +import org.apache.zookeeper.proto.GetDataRequest;
    +import org.apache.zookeeper.server.quorum.QuorumPeer;
    +import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer;
    +import org.apache.zookeeper.server.quorum.UpgradeableSessionTracker;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +
    +        String data = "test";
    +        String path = "/ephemeralcreatemultiop";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        QuorumZooKeeperServer server = getConnectedServer(zk.getSessionId());
    +        Assert.assertNotNull("unable to find server interlocutor", server);
    +        UpgradeableSessionTracker sessionTracker = (UpgradeableSessionTracker)server.getSessionTracker();
    +        Assert.assertFalse("session already global", sessionTracker.isGlobalSession(zk.getSessionId()));
    +
    +        List<OpResult> multi = null;
    +        try {
    +            multi = zk.multi(Arrays.asList(
    +                    Op.setData(path, data.getBytes(), 0),
    +                    Op.create(path + "/e", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL),
    +                    Op.create(path + "/p", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
    +                    Op.create(path + "/q", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
    +            ));
    +        } catch (KeeperException.SessionExpiredException e) {
    +            // the scenario that inspired this unit test
    +            Assert.fail("received session expired for a session promotion in a multi-op");
    +        }
    +
    +        Assert.assertNotNull(multi);
    +        Assert.assertEquals(4, multi.size());
    +        Assert.assertEquals(data, new String(zk.getData(path + "/e", false, null)));
    +        Assert.assertEquals(data, new String(zk.getData(path + "/p", false, null)));
    +        Assert.assertEquals(data, new String(zk.getData(path + "/q", false, null)));
    +        Assert.assertTrue("session not promoted", sessionTracker.isGlobalSession(zk.getSessionId()));
    +    }
    +
    +    @Test
    +    public void directCheckUpgradeSessionTest() throws IOException, InterruptedException, KeeperException {
    +        final ZooKeeper zk = createClient();
    +
    +        String path = "/directcheckupgradesession";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        QuorumZooKeeperServer server = getConnectedServer(zk.getSessionId());
    +        Assert.assertNotNull("unable to find server interlocutor", server);
    +
    +        Request readRequest = makeGetDataRequest(path, zk.getSessionId());
    +        Request createRequest = makeCreateRequest(path + "/e", zk.getSessionId());
    +        Assert.assertNull("tried to upgrade on a read", server.checkUpgradeSession(readRequest));
    +        Assert.assertNotNull("failed to upgrade on a create", server.checkUpgradeSession(createRequest));
    +        Assert.assertNull("tried to upgrade after successful promotion",
    +                server.checkUpgradeSession(createRequest));
    +    }
    +
    +    private Request makeGetDataRequest(String path, long sessionId) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    +        BinaryOutputArchive boa = BinaryOutputArchive.getArchive(boas);
    +        GetDataRequest getDataRequest = new GetDataRequest(path, false);
    +        getDataRequest.serialize(boa, "request");
    +        ByteBuffer bb = ByteBuffer.wrap(boas.toByteArray());
    +        return new Request(null, sessionId, 1, ZooDefs.OpCode.getData, bb, new ArrayList<>());
    --- End diff --
    
    `new ArrayList<>()` would not work in JDK 7 (the compiler can't infer the type w/o explicitly specify the type parameter here). I think you are using JDK 8 so you don't see this. Suggest change this to 'new ArrayList<Id>()' so it works with both JDK 7 and 8. Pre-commit job also failed because of this:  https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/462/console


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r106797374
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,87 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +        waitForClient(zk, ZooKeeper.States.CONNECTED);
    +
    +        String data = "test";
    +        String path = "/ephemeralcreatemultiop";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        List<OpResult> multi = null;
    +        LOG.info("RUNNING MULTI-OP");
    +        try {
    +            multi = zk.multi(Arrays.asList(
    +                    Op.setData(path, data.getBytes(), 0),
    +                    Op.create(path + "/e", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL),
    +                    Op.create(path + "/p", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
    +                    Op.create(path + "/q", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
    +            ));
    +        } catch (KeeperException.SessionExpiredException e) {
    +            LOG.info("SESSION EXPIRED", e);
    +        }
    +        LOG.info("TESTING RESULTS");
    +        Assert.assertNotNull(multi);
    --- End diff --
    
    In addition to these tests, it might be helpful to verify that a session upgrade has actually happened. I think we can verify that by comparing the old session id retrieved from zk before the multi-op and the new session id?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r106757640
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,87 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +        waitForClient(zk, ZooKeeper.States.CONNECTED);
    --- End diff --
    
    Not sure we need this. I don't think we need to do this in other tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by enixon <gi...@git.apache.org>.
Github user enixon commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r107191973
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,135 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.jute.BinaryOutputArchive;
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.proto.CreateRequest;
    +import org.apache.zookeeper.proto.GetDataRequest;
    +import org.apache.zookeeper.server.quorum.QuorumPeer;
    +import org.apache.zookeeper.server.quorum.QuorumZooKeeperServer;
    +import org.apache.zookeeper.server.quorum.UpgradeableSessionTracker;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +
    +        String data = "test";
    +        String path = "/ephemeralcreatemultiop";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        QuorumZooKeeperServer server = getConnectedServer(zk.getSessionId());
    +        Assert.assertNotNull("unable to find server interlocutor", server);
    +        UpgradeableSessionTracker sessionTracker = (UpgradeableSessionTracker)server.getSessionTracker();
    +        Assert.assertFalse("session already global", sessionTracker.isGlobalSession(zk.getSessionId()));
    +
    +        List<OpResult> multi = null;
    +        try {
    +            multi = zk.multi(Arrays.asList(
    +                    Op.setData(path, data.getBytes(), 0),
    +                    Op.create(path + "/e", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL),
    +                    Op.create(path + "/p", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT),
    +                    Op.create(path + "/q", data.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
    +            ));
    +        } catch (KeeperException.SessionExpiredException e) {
    +            // the scenario that inspired this unit test
    +            Assert.fail("received session expired for a session promotion in a multi-op");
    +        }
    +
    +        Assert.assertNotNull(multi);
    +        Assert.assertEquals(4, multi.size());
    +        Assert.assertEquals(data, new String(zk.getData(path + "/e", false, null)));
    +        Assert.assertEquals(data, new String(zk.getData(path + "/p", false, null)));
    +        Assert.assertEquals(data, new String(zk.getData(path + "/q", false, null)));
    +        Assert.assertTrue("session not promoted", sessionTracker.isGlobalSession(zk.getSessionId()));
    +    }
    +
    +    @Test
    +    public void directCheckUpgradeSessionTest() throws IOException, InterruptedException, KeeperException {
    +        final ZooKeeper zk = createClient();
    +
    +        String path = "/directcheckupgradesession";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        QuorumZooKeeperServer server = getConnectedServer(zk.getSessionId());
    +        Assert.assertNotNull("unable to find server interlocutor", server);
    +
    +        Request readRequest = makeGetDataRequest(path, zk.getSessionId());
    +        Request createRequest = makeCreateRequest(path + "/e", zk.getSessionId());
    +        Assert.assertNull("tried to upgrade on a read", server.checkUpgradeSession(readRequest));
    +        Assert.assertNotNull("failed to upgrade on a create", server.checkUpgradeSession(createRequest));
    +        Assert.assertNull("tried to upgrade after successful promotion",
    +                server.checkUpgradeSession(createRequest));
    +    }
    +
    +    private Request makeGetDataRequest(String path, long sessionId) throws IOException {
    +        ByteArrayOutputStream boas = new ByteArrayOutputStream();
    +        BinaryOutputArchive boa = BinaryOutputArchive.getArchive(boas);
    +        GetDataRequest getDataRequest = new GetDataRequest(path, false);
    +        getDataRequest.serialize(boa, "request");
    +        ByteBuffer bb = ByteBuffer.wrap(boas.toByteArray());
    +        return new Request(null, sessionId, 1, ZooDefs.OpCode.getData, bb, new ArrayList<>());
    --- End diff --
    
    Can't believe I missed those - something about them was bothering me about those lines when I was preparing the commit too.
    
    Will fix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r106759077
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,87 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    --- End diff --
    
    Would it be possible to test `public Request checkUpgradeSession(Request request)` directly in addition to this integration style test?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper issue #195: ZOOKEEPER-2725: Promote local session to global when e...

Posted by hanm <gi...@git.apache.org>.
Github user hanm commented on the issue:

    https://github.com/apache/zookeeper/pull/195
  
    LGTM, except it does not build with JDK 7. Will merge after the fix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] zookeeper pull request #195: ZOOKEEPER-2725: Promote local session to global...

Posted by afine <gi...@git.apache.org>.
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/195#discussion_r106757751
  
    --- Diff: src/java/test/org/apache/zookeeper/server/MultiOpSessionUpgradeTest.java ---
    @@ -0,0 +1,87 @@
    +/**
    + * 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.zookeeper.server;
    +
    +import org.apache.zookeeper.CreateMode;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Op;
    +import org.apache.zookeeper.OpResult;
    +import org.apache.zookeeper.ZooDefs;
    +import org.apache.zookeeper.ZooKeeper;
    +import org.apache.zookeeper.test.ClientBase;
    +import org.apache.zookeeper.test.QuorumBase;
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +public class MultiOpSessionUpgradeTest extends QuorumBase {
    +    protected static final Logger LOG = LoggerFactory.getLogger(MultiOpSessionUpgradeTest.class);
    +
    +    @Override
    +    public void setUp() throws Exception {
    +        localSessionsEnabled = true;
    +        localSessionsUpgradingEnabled = true;
    +        super.setUp();
    +    }
    +
    +    @Test
    +    public void ephemeralCreateMultiOpTest() throws KeeperException, InterruptedException, IOException {
    +        final ZooKeeper zk = createClient();
    +        waitForClient(zk, ZooKeeper.States.CONNECTED);
    +
    +        String data = "test";
    +        String path = "/ephemeralcreatemultiop";
    +        zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
    +
    +        List<OpResult> multi = null;
    +        LOG.info("RUNNING MULTI-OP");
    --- End diff --
    
    nit: not sure we need all of this logging


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---