You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by ifesdjeen <gi...@git.apache.org> on 2018/09/26 16:05:39 UTC

[GitHub] cassandra pull request #275: 14727

GitHub user ifesdjeen opened a pull request:

    https://github.com/apache/cassandra/pull/275

    14727

    

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

    $ git pull https://github.com/belliottsmith/cassandra 14727

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

    https://github.com/apache/cassandra/pull/275.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 #275
    
----
commit 150f4236f43593ed6a832ada5490c5b6ea64b88c
Author: Benedict Elliott Smith <be...@...>
Date:   2018-09-19T11:52:27Z

    Transient Replication support for EACH_QUORUM, and correction of behaviour for LOCAL_QUORUM

commit 3bd3f7b8ede99c69a810e75eff762deffdc8cf72
Author: Benedict Elliott Smith <be...@...>
Date:   2018-09-26T10:08:37Z

    circleci

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #275: 14727

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

    https://github.com/apache/cassandra/pull/275#discussion_r220628704
  
    --- Diff: test/unit/org/apache/cassandra/service/WriteResponseHandlerTransientTest.java ---
    @@ -173,9 +173,15 @@ public void checkPendingReplicasAreNotFiltered()
         private static void assertSpeculationReplicas(ReplicaPlan.ForTokenWrite expected, EndpointsForToken replicas, Predicate<InetAddressAndPort> livePredicate)
         {
             ReplicaPlan.ForTokenWrite actual = getSpeculationContext(replicas, livePredicate);
    -        Assert.assertTrue(Iterables.elementsEqual(expected.pending(), actual.pending()));
    -        Assert.assertTrue(Iterables.elementsEqual(expected.live(), actual.live()));
    -        Assert.assertTrue(Iterables.elementsEqual(expected.contacts(), actual.contacts()));
    +        assertEquals(expected.pending(), actual.pending());
    +        assertEquals(expected.live(), actual.live());
    +        assertEquals(expected.contacts(), actual.contacts());
    +    }
    +
    +    private static void assertEquals(ReplicaCollection<?> a, ReplicaCollection<?> b)
    +    {
    +        if (!Iterables.elementsEqual(a, b))
    +            Assert.assertTrue(a + " vs " + b, false);
    --- End diff --
    
    we might want to consolidate this with `assertMultimapEqualsIgnoreOrder`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #275: 14727

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

    https://github.com/apache/cassandra/pull/275#discussion_r220659112
  
    --- Diff: src/java/org/apache/cassandra/locator/ReplicaPlans.java ---
    @@ -265,17 +268,23 @@ E select(Keyspace keyspace, ConsistencyLevel consistencyLevel, L liveAndDown, L
                 if (!any(liveAndDown.all(), Replica::isTransient))
                     return liveAndDown.all();
     
    -            assert consistencyLevel != EACH_QUORUM;
    --- End diff --
    
    After this change `consistencyLevel` is unused in both cases.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] cassandra pull request #275: 14727

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

    https://github.com/apache/cassandra/pull/275#discussion_r220627686
  
    --- Diff: test/unit/org/apache/cassandra/locator/ReplicaPlansTest.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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.cassandra.locator;
    +
    +import com.google.common.base.Predicates;
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.ImmutableSet;
    +import org.apache.cassandra.config.DatabaseDescriptor;
    +import org.apache.cassandra.db.ConsistencyLevel;
    +import org.apache.cassandra.db.Keyspace;
    +import org.apache.cassandra.dht.Token;
    +import org.apache.cassandra.schema.KeyspaceMetadata;
    +import org.apache.cassandra.schema.KeyspaceParams;
    +import org.junit.Test;
    +
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.cassandra.locator.Replica.fullReplica;
    +import static org.apache.cassandra.locator.ReplicaUtils.*;
    +
    +public class ReplicaPlansTest
    +{
    +
    +    static
    +    {
    +        DatabaseDescriptor.daemonInitialization();
    +    }
    +
    +    static class Snitch extends AbstractNetworkTopologySnitch
    +    {
    +        final Set<InetAddressAndPort> dc1;
    +        Snitch(Set<InetAddressAndPort> dc1)
    +        {
    +            this.dc1 = dc1;
    +        }
    +        @Override
    +        public String getRack(InetAddressAndPort endpoint)
    +        {
    +            return dc1.contains(endpoint) ? "R1" : "R2";
    +        }
    +
    +        @Override
    +        public String getDatacenter(InetAddressAndPort endpoint)
    +        {
    +            return dc1.contains(endpoint) ? "DC1" : "DC2";
    +        }
    +    }
    +
    +    private static Keyspace ks(Set<InetAddressAndPort> dc1, Map<String, String> replication)
    +    {
    +        replication = ImmutableMap.<String, String>builder().putAll(replication).put("class", "NetworkTopologyStrategy").build();
    +        Keyspace keyspace = Keyspace.mockKS(KeyspaceMetadata.create("blah", KeyspaceParams.create(false, replication)));
    +        Snitch snitch = new Snitch(dc1);
    +        DatabaseDescriptor.setEndpointSnitch(snitch);
    +        keyspace.getReplicationStrategy().snitch = snitch;
    +        return keyspace;
    +    }
    +
    +    private static Replica full(InetAddressAndPort ep) { return fullReplica(ep, R1); }
    +
    +
    +
    +    @Test
    +    public void testWriteEachQuorum()
    +    {
    +        IEndpointSnitch stash = DatabaseDescriptor.getEndpointSnitch();
    +        final Token token = tk(1L);
    +        try
    +        {
    +            {
    +                // all full natural
    +                Keyspace ks = ks(ImmutableSet.of(EP1, EP2, EP3), ImmutableMap.of("DC1", "3", "DC2", "3"));
    +                EndpointsForToken natural = EndpointsForToken.of(token, full(EP1), full(EP2), full(EP3), full(EP4), full(EP5), full(EP6));
    +                EndpointsForToken pending = EndpointsForToken.empty(token);
    +                ReplicaPlan.ForTokenWrite plan = ReplicaPlans.forWrite(ks, ConsistencyLevel.EACH_QUORUM, natural, pending, Predicates.alwaysTrue(), ReplicaPlans.writeNormal);
    +                assertEquals(natural, plan.liveAndDown);
    +                assertEquals(natural, plan.live);
    +                assertEquals(natural, plan.contacts());
    +            }
    +            {
    +                // all natural and up, one transient in each DC
    +                Keyspace ks = ks(ImmutableSet.of(EP1, EP2, EP3), ImmutableMap.of("DC1", "3", "DC2", "3"));
    +                EndpointsForToken natural = EndpointsForToken.of(token, full(EP1), full(EP2), trans(EP3), full(EP4), full(EP5), trans(EP6));
    +                EndpointsForToken pending = EndpointsForToken.empty(token);
    +                ReplicaPlan.ForTokenWrite plan = ReplicaPlans.forWrite(ks, ConsistencyLevel.EACH_QUORUM, natural, pending, Predicates.alwaysTrue(), ReplicaPlans.writeNormal);
    +                assertEquals(natural, plan.liveAndDown);
    +                assertEquals(natural, plan.live);
    +                EndpointsForToken expectContacts = EndpointsForToken.of(token, full(EP1), full(EP2), full(EP4), full(EP5));
    +                assertEquals(expectContacts, plan.contacts());
    +            }
    +        }
    +        finally
    +        {
    +            DatabaseDescriptor.setEndpointSnitch(stash);
    +        }
    +
    +        {
    +            // test simple
    --- End diff --
    
    Was this intended to be tested? If not we might want to remove it.
    
    Might also make sense to split test cases as they seem to be purposely isolated.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org