cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jmcken...@apache.org
Subject [3/3] git commit: Merge branch 'cassandra-2.1' into trunk
Date Mon, 03 Nov 2014 19:10:02 GMT
Merge branch 'cassandra-2.1' into trunk

Conflicts:
	src/java/org/apache/cassandra/cql/QueryProcessor.java
	src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/49768fea
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/49768fea
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/49768fea

Branch: refs/heads/trunk
Commit: 49768fea0071bd2f084c70c9b87468e2f48eb2a5
Parents: 632ebc5 ad84e4d
Author: Joshua McKenzie <jmckenzie@apache.org>
Authored: Mon Nov 3 13:06:25 2014 -0600
Committer: Joshua McKenzie <jmckenzie@apache.org>
Committed: Mon Nov 3 13:06:25 2014 -0600

----------------------------------------------------------------------
 .../org/apache/cassandra/client/RingCache.java  |   8 +-
 .../cassandra/config/DatabaseDescriptor.java    |   6 +-
 .../cql3/statements/SelectStatement.java        |   4 +-
 .../apache/cassandra/db/BatchlogManager.java    |   2 +-
 .../cassandra/db/HintedHandOffManager.java      |   2 +-
 .../db/marshal/LocalByPartionerType.java        |   7 +-
 .../apache/cassandra/dht/AbstractBounds.java    |   4 +-
 .../dht/AbstractByteOrderedPartitioner.java     |  20 +-
 .../cassandra/dht/AbstractPartitioner.java      |   5 +-
 .../org/apache/cassandra/dht/AbstractToken.java |  53 +++++
 .../apache/cassandra/dht/BigIntegerToken.java   |   6 +-
 src/java/org/apache/cassandra/dht/Bounds.java   |   6 +-
 .../cassandra/dht/ByteOrderedPartitioner.java   |   4 +-
 .../org/apache/cassandra/dht/BytesToken.java    |   5 +-
 .../apache/cassandra/dht/ExcludingBounds.java   |   4 +-
 .../org/apache/cassandra/dht/IPartitioner.java  |  12 +-
 .../cassandra/dht/IncludingExcludingBounds.java |   6 +-
 .../apache/cassandra/dht/LocalPartitioner.java  |   6 +-
 .../org/apache/cassandra/dht/LocalToken.java    |   6 +-
 .../org/apache/cassandra/dht/LongToken.java     |   6 +-
 .../cassandra/dht/Murmur3Partitioner.java       |  25 +-
 .../dht/OrderPreservingPartitioner.java         |  20 +-
 .../apache/cassandra/dht/RandomPartitioner.java |  18 +-
 src/java/org/apache/cassandra/dht/Range.java    |  39 ++--
 .../org/apache/cassandra/dht/RangeStreamer.java |   2 +-
 .../org/apache/cassandra/dht/RingPosition.java  |   2 +-
 .../org/apache/cassandra/dht/StringToken.java   |   6 +-
 src/java/org/apache/cassandra/dht/Token.java    |  52 +----
 .../apache/cassandra/gms/TokenSerializer.java   |   2 +-
 .../apache/cassandra/service/StorageProxy.java  |   2 +-
 .../cassandra/thrift/CassandraServer.java       |   4 +-
 .../apache/cassandra/tools/SSTableExport.java   |   2 +-
 .../apache/cassandra/tools/SSTableImport.java   |   6 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     |   2 +-
 .../apache/cassandra/db/KeyCollisionTest.java   | 230 -------------------
 .../apache/cassandra/dht/BootStrapperTest.java  |   2 +-
 .../dht/ByteOrderedPartitionerTest.java         |   2 +-
 .../apache/cassandra/dht/KeyCollisionTest.java  | 230 +++++++++++++++++++
 .../cassandra/dht/Murmur3PartitionerTest.java   |   4 +-
 .../dht/OrderPreservingPartitionerTest.java     |   2 +-
 .../cassandra/dht/PartitionerTestCase.java      |  12 +-
 .../cassandra/dht/RandomPartitionerTest.java    |   2 +-
 .../org/apache/cassandra/dht/RangeTest.java     |   4 +-
 43 files changed, 441 insertions(+), 401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/dht/BytesToken.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/tools/SSTableExport.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/49768fea/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
index 0000000,e143f30..aaa3bc9
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
@@@ -1,0 -1,217 +1,230 @@@
+ /**
+  * 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.dht;
+ 
+ import java.math.BigInteger;
+ import java.nio.ByteBuffer;
+ import java.util.*;
+ 
++import org.junit.BeforeClass;
+ import org.junit.Test;
+ import org.apache.cassandra.SchemaLoader;
+ import org.apache.cassandra.Util;
+ import org.apache.cassandra.db.*;
+ import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+ import org.apache.cassandra.db.marshal.AbstractType;
+ import org.apache.cassandra.db.marshal.IntegerType;
+ import org.apache.cassandra.config.*;
++import org.apache.cassandra.exceptions.ConfigurationException;
++import org.apache.cassandra.locator.SimpleStrategy;
+ import org.apache.cassandra.service.StorageService;
+ import org.apache.cassandra.utils.*;
+ 
+ import static org.apache.cassandra.Util.dk;
+ 
+ 
+ /**
+  * Test cases where multiple keys collides, ie have the same token.
+  * Order preserving partitioner have no possible collision and creating
+  * collision for the RandomPartitioner is ... difficult, so we create a dumb
+  * length partitioner that takes the length of the key as token, making
+  * collision easy and predictable.
+  */
 -public class KeyCollisionTest extends SchemaLoader
++public class KeyCollisionTest
+ {
+     IPartitioner oldPartitioner;
 -    private static final String KEYSPACE = "Keyspace1";
++    private static final String KEYSPACE1 = "KeyCollisionTest1";
+     private static final String CF = "Standard1";
+ 
++    @BeforeClass
++    public static void defineSchema() throws ConfigurationException
++    {
++        SchemaLoader.prepareServer();
++        SchemaLoader.createKeyspace(KEYSPACE1,
++                                    SimpleStrategy.class,
++                                    KSMetaData.optsWithRF(1),
++                                    SchemaLoader.standardCFMD(KEYSPACE1, CF));
++    }
++
+     protected void setUp()
+     {
+         oldPartitioner = DatabaseDescriptor.getPartitioner();
+         DatabaseDescriptor.setPartitioner(new LengthPartitioner());
+     }
+ 
+     protected void tearDown()
+     {
+         DatabaseDescriptor.setPartitioner(oldPartitioner);
+     }
+ 
+     @Test
+     public void testGetSliceWithCollision() throws Exception
+     {
 -        Keyspace keyspace = Keyspace.open(KEYSPACE);
++        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+         cfs.clearUnsafe();
+ 
+         insert("k1", "k2", "k3");       // token = 2
+         insert("key1", "key2", "key3"); // token = 4
+         insert("longKey1", "longKey2"); // token = 8
+ 
+         List<Row> rows = cfs.getRangeSlice(new Bounds<RowPosition>(dk("k2"),
dk("key2")), null, new IdentityQueryFilter(), 10000);
+         assert rows.size() == 4 : "Expecting 4 keys, got " + rows.size();
+         assert rows.get(0).key.getKey().equals(ByteBufferUtil.bytes("k2"));
+         assert rows.get(1).key.getKey().equals(ByteBufferUtil.bytes("k3"));
+         assert rows.get(2).key.getKey().equals(ByteBufferUtil.bytes("key1"));
+         assert rows.get(3).key.getKey().equals(ByteBufferUtil.bytes("key2"));
+     }
+ 
+     private void insert(String... keys)
+     {
+         for (String key : keys)
+             insert(key);
+     }
+ 
+     private void insert(String key)
+     {
+         Mutation rm;
 -        rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(key));
++        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key));
+         rm.add(CF, Util.cellname("column"), ByteBufferUtil.bytes("asdf"), 0);
 -        rm.apply();
++        rm.applyUnsafe();
+     }
+ 
+     public static class LengthPartitioner extends AbstractPartitioner
+     {
+         public static final BigInteger ZERO = new BigInteger("0");
+         public static final BigIntegerToken MINIMUM = new BigIntegerToken("-1");
+ 
+         public DecoratedKey decorateKey(ByteBuffer key)
+         {
+             return new BufferDecoratedKey(getToken(key), key);
+         }
+ 
+         public BigIntegerToken midpoint(Token ltoken, Token rtoken)
+         {
+             // the symbolic MINIMUM token should act as ZERO: the empty bit array
+             BigInteger left = ltoken.equals(MINIMUM) ? ZERO : ((BigIntegerToken)ltoken).token;
+             BigInteger right = rtoken.equals(MINIMUM) ? ZERO : ((BigIntegerToken)rtoken).token;
+             Pair<BigInteger,Boolean> midpair = FBUtilities.midpoint(left, right, 127);
+             // discard the remainder
+             return new BigIntegerToken(midpair.left);
+         }
+ 
+         public BigIntegerToken getMinimumToken()
+         {
+             return MINIMUM;
+         }
+ 
+         public BigIntegerToken getRandomToken()
+         {
+             return new BigIntegerToken(BigInteger.valueOf(new Random().nextInt(15)));
+         }
+ 
+         private final Token.TokenFactory tokenFactory = new Token.TokenFactory() {
+             public ByteBuffer toByteArray(Token token)
+             {
+                 BigIntegerToken bigIntegerToken = (BigIntegerToken) token;
+                 return ByteBuffer.wrap(bigIntegerToken.token.toByteArray());
+             }
+ 
+             public Token fromByteArray(ByteBuffer bytes)
+             {
+                 return new BigIntegerToken(new BigInteger(ByteBufferUtil.getArray(bytes)));
+             }
+ 
+             public String toString(Token token)
+             {
+                 BigIntegerToken bigIntegerToken = (BigIntegerToken) token;
+                 return bigIntegerToken.token.toString();
+             }
+ 
+             public Token fromString(String string)
+             {
+                 return new BigIntegerToken(new BigInteger(string));
+             }
+ 
+             public void validate(String token) {}
+         };
+ 
+         public Token.TokenFactory getTokenFactory()
+         {
+             return tokenFactory;
+         }
+ 
+         public boolean preservesOrder()
+         {
+             return false;
+         }
+ 
+         public BigIntegerToken getToken(ByteBuffer key)
+         {
+             if (key.remaining() == 0)
+                 return MINIMUM;
+             return new BigIntegerToken(BigInteger.valueOf(key.remaining()));
+         }
+ 
+         @Override
+         public long getHeapSizeOf(Token token)
+         {
+             return 0;
+         }
+ 
+         public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
+         {
+             // allTokens will contain the count and be returned, sorted_ranges is shorthand
for token<->token math.
+             Map<Token, Float> allTokens = new HashMap<Token, Float>();
+             List<Range<Token>> sortedRanges = new ArrayList<Range<Token>>();
+ 
+             // this initializes the counts to 0 and calcs the ranges in order.
+             Token lastToken = sortedTokens.get(sortedTokens.size() - 1);
+             for (Token node : sortedTokens)
+             {
+                 allTokens.put(node, new Float(0.0));
+                 sortedRanges.add(new Range<Token>(lastToken, node));
+                 lastToken = node;
+             }
+ 
+             for (String ks : Schema.instance.getKeyspaces())
+             {
+                 for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values())
+                 {
+                     for (Range<Token> r : sortedRanges)
+                     {
+                         // Looping over every KS:CF:Range, get the splits size and add it
to the count
+                         allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks,
cfmd.cfName, r, 1).size());
+                     }
+                 }
+             }
+ 
+             // Sum every count up and divide count/total for the fractional ownership.
+             Float total = new Float(0.0);
+             for (Float f : allTokens.values())
+                 total += f;
+             for (Map.Entry<Token, Float> row : allTokens.entrySet())
+                 allTokens.put(row.getKey(), row.getValue() / total);
+ 
+             return allTokens;
+         }
+ 
+         public AbstractType<?> getTokenValidator()
+         {
+             return IntegerType.instance;
+         }
+     }
+ }


Mime
View raw message