You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ca...@apache.org on 2016/06/15 14:59:52 UTC

[19/20] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/src/java/org/apache/cassandra/tools/LoaderOptions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/LoaderOptions.java
index 91b3e8b,0000000..28d7bce
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/LoaderOptions.java
+++ b/src/java/org/apache/cassandra/tools/LoaderOptions.java
@@@ -1,543 -1,0 +1,563 @@@
++/*
++ *
++ * 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.tools;
 +
 +import java.io.File;
 +import java.lang.reflect.Constructor;
 +import java.lang.reflect.InvocationTargetException;
 +import java.net.*;
 +import java.util.HashSet;
 +import java.util.Set;
 +
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.exceptions.ConfigurationException;
 +import org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 +
 +import com.datastax.driver.core.AuthProvider;
 +import com.datastax.driver.core.PlainTextAuthProvider;
 +import org.apache.commons.cli.*;
 +
 +public class LoaderOptions
 +{
 +
 +    public static final String HELP_OPTION = "help";
 +    public static final String VERBOSE_OPTION = "verbose";
 +    public static final String NOPROGRESS_OPTION = "no-progress";
 +    public static final String NATIVE_PORT_OPTION = "port";
 +    public static final String USER_OPTION = "username";
 +    public static final String PASSWD_OPTION = "password";
 +    public static final String AUTH_PROVIDER_OPTION = "auth-provider";
 +    public static final String INITIAL_HOST_ADDRESS_OPTION = "nodes";
 +    public static final String IGNORE_NODES_OPTION = "ignore";
 +    public static final String CONNECTIONS_PER_HOST = "connections-per-host";
 +    public static final String CONFIG_PATH = "conf-path";
 +    public static final String THROTTLE_MBITS = "throttle";
 +    public static final String INTER_DC_THROTTLE_MBITS = "inter-dc-throttle";
 +    public static final String TOOL_NAME = "sstableloader";
 +
 +    /* client encryption options */
 +    public static final String SSL_TRUSTSTORE = "truststore";
 +    public static final String SSL_TRUSTSTORE_PW = "truststore-password";
 +    public static final String SSL_KEYSTORE = "keystore";
 +    public static final String SSL_KEYSTORE_PW = "keystore-password";
 +    public static final String SSL_PROTOCOL = "ssl-protocol";
 +    public static final String SSL_ALGORITHM = "ssl-alg";
 +    public static final String SSL_STORE_TYPE = "store-type";
 +    public static final String SSL_CIPHER_SUITES = "ssl-ciphers";
 +
 +    public final File directory;
 +    public final boolean debug;
 +    public final boolean verbose;
 +    public final boolean noProgress;
 +    public final int nativePort;
 +    public final String user;
 +    public final String passwd;
 +    public final AuthProvider authProvider;
 +    public final int throttle;
 +    public final int interDcThrottle;
 +    public final int storagePort;
 +    public final int sslStoragePort;
 +    public final EncryptionOptions.ClientEncryptionOptions clientEncOptions;
 +    public final int connectionsPerHost;
 +    public final EncryptionOptions.ServerEncryptionOptions serverEncOptions;
 +    public final Set<InetAddress> hosts;
 +    public final Set<InetAddress> ignores = new HashSet<>();
 +
 +    LoaderOptions(Builder builder)
 +    {
 +        directory = builder.directory;
 +        debug = builder.debug;
 +        verbose = builder.verbose;
 +        noProgress = builder.noProgress;
 +        nativePort = builder.nativePort;
 +        user = builder.user;
 +        passwd = builder.passwd;
 +        authProvider = builder.authProvider;
 +        throttle = builder.throttle;
 +        interDcThrottle = builder.interDcThrottle;
 +        storagePort = builder.storagePort;
 +        sslStoragePort = builder.sslStoragePort;
 +        clientEncOptions = builder.clientEncOptions;
 +        connectionsPerHost = builder.connectionsPerHost;
 +        serverEncOptions = builder.serverEncOptions;
 +        hosts = builder.hosts;
 +    }
 +
 +    static class Builder
 +    {
 +        File directory;
 +        boolean debug;
 +        boolean verbose;
 +        boolean noProgress;
 +        int nativePort = 9042;
 +        String user;
 +        String passwd;
 +        String authProviderName;
 +        AuthProvider authProvider;
 +        int throttle = 0;
 +        int interDcThrottle = 0;
 +        int storagePort;
 +        int sslStoragePort;
 +        EncryptionOptions.ClientEncryptionOptions clientEncOptions = new EncryptionOptions.ClientEncryptionOptions();
 +        int connectionsPerHost = 1;
 +        EncryptionOptions.ServerEncryptionOptions serverEncOptions = new EncryptionOptions.ServerEncryptionOptions();
 +        Set<InetAddress> hosts = new HashSet<>();
 +        Set<InetAddress> ignores = new HashSet<>();
 +
 +        Builder()
 +        {
 +            //
 +        }
 +
 +        public LoaderOptions build()
 +        {
 +            constructAuthProvider();
 +            return new LoaderOptions(this);
 +        }
 +
 +        public Builder directory(File directory)
 +        {
 +            this.directory = directory;
 +            return this;
 +        }
 +
 +        public Builder debug(boolean debug)
 +        {
 +            this.debug = debug;
 +            return this;
 +        }
 +
 +        public Builder verbose(boolean verbose)
 +        {
 +            this.verbose = verbose;
 +            return this;
 +        }
 +
 +        public Builder noProgress(boolean noProgress)
 +        {
 +            this.noProgress = noProgress;
 +            return this;
 +        }
 +
 +        public Builder nativePort(int nativePort)
 +        {
 +            this.nativePort = nativePort;
 +            return this;
 +        }
 +
 +        public Builder user(String user)
 +        {
 +            this.user = user;
 +            return this;
 +        }
 +
 +        public Builder password(String passwd)
 +        {
 +            this.passwd = passwd;
 +            return this;
 +        }
 +
 +        public Builder authProvider(AuthProvider authProvider)
 +        {
 +            this.authProvider = authProvider;
 +            return this;
 +        }
 +
 +        public Builder throttle(int throttle)
 +        {
 +            this.throttle = throttle;
 +            return this;
 +        }
 +
 +        public Builder interDcThrottle(int interDcThrottle)
 +        {
 +            this.interDcThrottle = interDcThrottle;
 +            return this;
 +        }
 +
 +        public Builder storagePort(int storagePort)
 +        {
 +            this.storagePort = storagePort;
 +            return this;
 +        }
 +
 +        public Builder sslStoragePort(int sslStoragePort)
 +        {
 +            this.sslStoragePort = sslStoragePort;
 +            return this;
 +        }
 +
 +        public Builder encOptions(EncryptionOptions.ClientEncryptionOptions encOptions)
 +        {
 +            this.clientEncOptions = encOptions;
 +            return this;
 +        }
 +
 +        public Builder connectionsPerHost(int connectionsPerHost)
 +        {
 +            this.connectionsPerHost = connectionsPerHost;
 +            return this;
 +        }
 +
 +        public Builder serverEncOptions(EncryptionOptions.ServerEncryptionOptions serverEncOptions)
 +        {
 +            this.serverEncOptions = serverEncOptions;
 +            return this;
 +        }
 +
 +        public Builder hosts(Set<InetAddress> hosts)
 +        {
 +            this.hosts = hosts;
 +            return this;
 +        }
 +
 +        public Builder host(InetAddress host)
 +        {
 +            hosts.add(host);
 +            return this;
 +        }
 +
 +        public Builder ignore(Set<InetAddress> ignores)
 +        {
 +            this.ignores = ignores;
 +            return this;
 +        }
 +
 +        public Builder ignore(InetAddress ignore)
 +        {
 +            ignores.add(ignore);
 +            return this;
 +        }
 +
 +        public Builder parseArgs(String cmdArgs[])
 +        {
 +            CommandLineParser parser = new GnuParser();
 +            CmdLineOptions options = getCmdLineOptions();
 +            try
 +            {
 +                CommandLine cmd = parser.parse(options, cmdArgs, false);
 +
 +                if (cmd.hasOption(HELP_OPTION))
 +                {
 +                    printUsage(options);
 +                    System.exit(0);
 +                }
 +
 +                String[] args = cmd.getArgs();
 +                if (args.length == 0)
 +                {
 +                    System.err.println("Missing sstable directory argument");
 +                    printUsage(options);
 +                    System.exit(1);
 +                }
 +
 +                if (args.length > 1)
 +                {
 +                    System.err.println("Too many arguments");
 +                    printUsage(options);
 +                    System.exit(1);
 +                }
 +
 +                String dirname = args[0];
 +                File dir = new File(dirname);
 +
 +                if (!dir.exists())
 +                {
 +                    errorMsg("Unknown directory: " + dirname, options);
 +                }
 +
 +                if (!dir.isDirectory())
 +                {
 +                    errorMsg(dirname + " is not a directory", options);
 +                }
 +
 +                directory = dir;
 +
 +                verbose = cmd.hasOption(VERBOSE_OPTION);
 +                noProgress = cmd.hasOption(NOPROGRESS_OPTION);
 +
 +                if (cmd.hasOption(NATIVE_PORT_OPTION))
 +                {
 +                    nativePort = Integer.parseInt(cmd.getOptionValue(NATIVE_PORT_OPTION));
 +                }
 +
 +                if (cmd.hasOption(USER_OPTION))
 +                {
 +                    user = cmd.getOptionValue(USER_OPTION);
 +                }
 +
 +                if (cmd.hasOption(PASSWD_OPTION))
 +                {
 +                    passwd = cmd.getOptionValue(PASSWD_OPTION);
 +                }
 +
 +                if (cmd.hasOption(AUTH_PROVIDER_OPTION))
 +                {
 +                    authProviderName = cmd.getOptionValue(AUTH_PROVIDER_OPTION);
 +                }
 +
 +                if (cmd.hasOption(INITIAL_HOST_ADDRESS_OPTION))
 +                {
 +                    String[] nodes = cmd.getOptionValue(INITIAL_HOST_ADDRESS_OPTION).split(",");
 +                    try
 +                    {
 +                        for (String node : nodes)
 +                        {
 +                            hosts.add(InetAddress.getByName(node.trim()));
 +                        }
 +                    } catch (UnknownHostException e)
 +                    {
 +                        errorMsg("Unknown host: " + e.getMessage(), options);
 +                    }
 +
 +                } else
 +                {
 +                    System.err.println("Initial hosts must be specified (-d)");
 +                    printUsage(options);
 +                    System.exit(1);
 +                }
 +
 +                if (cmd.hasOption(IGNORE_NODES_OPTION))
 +                {
 +                    String[] nodes = cmd.getOptionValue(IGNORE_NODES_OPTION).split(",");
 +                    try
 +                    {
 +                        for (String node : nodes)
 +                        {
 +                            ignores.add(InetAddress.getByName(node.trim()));
 +                        }
 +                    } catch (UnknownHostException e)
 +                    {
 +                        errorMsg("Unknown host: " + e.getMessage(), options);
 +                    }
 +                }
 +
 +                if (cmd.hasOption(CONNECTIONS_PER_HOST))
 +                {
 +                    connectionsPerHost = Integer.parseInt(cmd.getOptionValue(CONNECTIONS_PER_HOST));
 +                }
 +
 +                // try to load config file first, so that values can be
 +                // rewritten with other option values.
 +                // otherwise use default config.
 +                Config config;
 +                if (cmd.hasOption(CONFIG_PATH))
 +                {
 +                    File configFile = new File(cmd.getOptionValue(CONFIG_PATH));
 +                    if (!configFile.exists())
 +                    {
 +                        errorMsg("Config file not found", options);
 +                    }
 +                    config = new YamlConfigurationLoader().loadConfig(configFile.toURI().toURL());
 +                }
 +                else
 +                {
 +                    config = new Config();
 +                    // unthrottle stream by default
 +                    config.stream_throughput_outbound_megabits_per_sec = 0;
 +                    config.inter_dc_stream_throughput_outbound_megabits_per_sec = 0;
 +                }
 +                storagePort = config.storage_port;
 +                sslStoragePort = config.ssl_storage_port;
 +                throttle = config.stream_throughput_outbound_megabits_per_sec;
 +                clientEncOptions = config.client_encryption_options;
 +                serverEncOptions = config.server_encryption_options;
 +
 +                if (cmd.hasOption(THROTTLE_MBITS))
 +                {
 +                    throttle = Integer.parseInt(cmd.getOptionValue(THROTTLE_MBITS));
 +                }
 +
 +                if (cmd.hasOption(INTER_DC_THROTTLE_MBITS))
 +                {
 +                    interDcThrottle = Integer.parseInt(cmd.getOptionValue(INTER_DC_THROTTLE_MBITS));
 +                }
 +
 +                if (cmd.hasOption(SSL_TRUSTSTORE) || cmd.hasOption(SSL_TRUSTSTORE_PW) ||
 +                            cmd.hasOption(SSL_KEYSTORE) || cmd.hasOption(SSL_KEYSTORE_PW))
 +                {
 +                    clientEncOptions.enabled = true;
 +                }
 +
 +                if (cmd.hasOption(SSL_TRUSTSTORE))
 +                {
 +                    clientEncOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE);
 +                }
 +
 +                if (cmd.hasOption(SSL_TRUSTSTORE_PW))
 +                {
 +                    clientEncOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW);
 +                }
 +
 +                if (cmd.hasOption(SSL_KEYSTORE))
 +                {
 +                    clientEncOptions.keystore = cmd.getOptionValue(SSL_KEYSTORE);
 +                    // if a keystore was provided, lets assume we'll need to use
 +                    // it
 +                    clientEncOptions.require_client_auth = true;
 +                }
 +
 +                if (cmd.hasOption(SSL_KEYSTORE_PW))
 +                {
 +                    clientEncOptions.keystore_password = cmd.getOptionValue(SSL_KEYSTORE_PW);
 +                }
 +
 +                if (cmd.hasOption(SSL_PROTOCOL))
 +                {
 +                    clientEncOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL);
 +                }
 +
 +                if (cmd.hasOption(SSL_ALGORITHM))
 +                {
 +                    clientEncOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM);
 +                }
 +
 +                if (cmd.hasOption(SSL_STORE_TYPE))
 +                {
 +                    clientEncOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE);
 +                }
 +
 +                if (cmd.hasOption(SSL_CIPHER_SUITES))
 +                {
 +                    clientEncOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(",");
 +                }
 +
 +                return this;
 +            }
 +            catch (ParseException | ConfigurationException | MalformedURLException e)
 +            {
 +                errorMsg(e.getMessage(), options);
 +                return null;
 +            }
 +        }
 +
 +        private void constructAuthProvider()
 +        {
 +            // Both username and password need to be provided
 +            if ((user != null) != (passwd != null))
 +                errorMsg("Username and password must both be provided", getCmdLineOptions());
 +
 +            if (user != null)
 +            {
 +                // Support for 3rd party auth providers that support plain text credentials.
 +                // In this case the auth provider must provide a constructor of the form:
 +                //
 +                // public MyAuthProvider(String username, String password)
 +                if (authProviderName != null)
 +                {
 +                    try
 +                    {
 +                        Class authProviderClass = Class.forName(authProviderName);
 +                        Constructor constructor = authProviderClass.getConstructor(String.class, String.class);
 +                        authProvider = (AuthProvider)constructor.newInstance(user, passwd);
 +                    }
 +                    catch (ClassNotFoundException e)
 +                    {
 +                        errorMsg("Unknown auth provider: " + e.getMessage(), getCmdLineOptions());
 +                    }
 +                    catch (NoSuchMethodException e)
 +                    {
 +                        errorMsg("Auth provider does not support plain text credentials: " + e.getMessage(), getCmdLineOptions());
 +                    }
 +                    catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e)
 +                    {
 +                        errorMsg("Could not create auth provider with plain text credentials: " + e.getMessage(), getCmdLineOptions());
 +                    }
 +                }
 +                else
 +                {
 +                    // If a 3rd party auth provider wasn't provided use the driver plain text provider
 +                    this.authProvider = new PlainTextAuthProvider(user, passwd);
 +                }
 +            }
 +            // Alternate support for 3rd party auth providers that don't use plain text credentials.
 +            // In this case the auth provider must provide a nullary constructor of the form:
 +            //
 +            // public MyAuthProvider()
 +            else if (authProviderName != null)
 +            {
 +                try
 +                {
 +                    authProvider = (AuthProvider)Class.forName(authProviderName).newInstance();
 +                }
 +                catch (ClassNotFoundException | InstantiationException | IllegalAccessException e)
 +                {
 +                    errorMsg("Unknown auth provider: " + e.getMessage(), getCmdLineOptions());
 +                }
 +            }
 +        }
 +    }
 +
 +    public static Builder builder()
 +    {
 +        return new Builder();
 +    }
 +
 +    private static void errorMsg(String msg, CmdLineOptions options)
 +    {
 +        System.err.println(msg);
 +        printUsage(options);
 +        System.exit(1);
 +    }
 +
 +    private static CmdLineOptions getCmdLineOptions()
 +    {
 +        CmdLineOptions options = new CmdLineOptions();
 +        options.addOption("v", VERBOSE_OPTION, "verbose output");
 +        options.addOption("h", HELP_OPTION, "display this help message");
 +        options.addOption(null, NOPROGRESS_OPTION, "don't display progress");
 +        options.addOption("i", IGNORE_NODES_OPTION, "NODES", "don't stream to this (comma separated) list of nodes");
 +        options.addOption("d", INITIAL_HOST_ADDRESS_OPTION, "initial hosts", "Required. try to connect to these hosts (comma separated) initially for ring information");
 +        options.addOption("p", NATIVE_PORT_OPTION, "rpc port", "port used for native connection (default 9042)");
 +        options.addOption("t", THROTTLE_MBITS, "throttle", "throttle speed in Mbits (default unlimited)");
 +        options.addOption("idct", INTER_DC_THROTTLE_MBITS, "inter-dc-throttle", "inter-datacenter throttle speed in Mbits (default unlimited)");
 +        options.addOption("u", USER_OPTION, "username", "username for cassandra authentication");
 +        options.addOption("pw", PASSWD_OPTION, "password", "password for cassandra authentication");
 +        options.addOption("ap", AUTH_PROVIDER_OPTION, "auth provider", "custom AuthProvider class name for cassandra authentication");
 +        options.addOption("cph", CONNECTIONS_PER_HOST, "connectionsPerHost", "number of concurrent connections-per-host.");
 +        // ssl connection-related options
 +        options.addOption("ts", SSL_TRUSTSTORE, "TRUSTSTORE", "Client SSL: full path to truststore");
 +        options.addOption("tspw", SSL_TRUSTSTORE_PW, "TRUSTSTORE-PASSWORD", "Client SSL: password of the truststore");
 +        options.addOption("ks", SSL_KEYSTORE, "KEYSTORE", "Client SSL: full path to keystore");
 +        options.addOption("kspw", SSL_KEYSTORE_PW, "KEYSTORE-PASSWORD", "Client SSL: password of the keystore");
 +        options.addOption("prtcl", SSL_PROTOCOL, "PROTOCOL", "Client SSL: connections protocol to use (default: TLS)");
 +        options.addOption("alg", SSL_ALGORITHM, "ALGORITHM", "Client SSL: algorithm (default: SunX509)");
 +        options.addOption("st", SSL_STORE_TYPE, "STORE-TYPE", "Client SSL: type of store");
 +        options.addOption("ciphers", SSL_CIPHER_SUITES, "CIPHER-SUITES", "Client SSL: comma-separated list of encryption suites to use");
 +        options.addOption("f", CONFIG_PATH, "path to config file", "cassandra.yaml file path for streaming throughput and client/server SSL.");
 +        return options;
 +    }
 +
 +    public static void printUsage(Options options)
 +    {
 +        String usage = String.format("%s [options] <dir_path>", TOOL_NAME);
 +        String header = System.lineSeparator() +
 +                "Bulk load the sstables found in the directory <dir_path> to the configured cluster." +
 +                "The parent directories of <dir_path> are used as the target keyspace/table name. " +
 +                "So for instance, to load an sstable named Standard1-g-1-Data.db into Keyspace1/Standard1, " +
 +                "you will need to have the files Standard1-g-1-Data.db and Standard1-g-1-Index.db into a directory /path/to/Keyspace1/Standard1/.";
 +        String footer = System.lineSeparator() +
 +                "You can provide cassandra.yaml file with -f command line option to set up streaming throughput, client and server encryption options. " +
 +                "Only stream_throughput_outbound_megabits_per_sec, server_encryption_options and client_encryption_options are read from yaml. " +
 +                "You can override options read from cassandra.yaml with corresponding command line options.";
 +        new HelpFormatter().printHelp(usage, header, options, footer);
 +    }
- }
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/src/java/org/apache/cassandra/utils/OverlapIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/OverlapIterator.java
index 131a749,b346a62..7c1544a
--- a/src/java/org/apache/cassandra/utils/OverlapIterator.java
+++ b/src/java/org/apache/cassandra/utils/OverlapIterator.java
@@@ -1,3 -1,23 +1,23 @@@
+ /*
+  *
+  * 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.utils;
  
  import java.util.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/src/java/org/apache/cassandra/utils/RMIServerSocketFactoryImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
index b53c2a0,0000000..8345118
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
+++ b/src/java/org/apache/cassandra/utils/memory/EnsureOnHeap.java
@@@ -1,150 -1,0 +1,170 @@@
++/*
++ *
++ * 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.utils.memory;
 +
 +import java.util.Iterator;
 +
 +import org.apache.cassandra.db.BufferDecoratedKey;
 +import org.apache.cassandra.db.Clustering;
 +import org.apache.cassandra.db.DecoratedKey;
 +import org.apache.cassandra.db.DeletionInfo;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.db.transform.Transformation;
 +import org.apache.cassandra.utils.SearchIterator;
 +
 +public abstract class EnsureOnHeap extends Transformation
 +{
 +    public abstract DecoratedKey applyToPartitionKey(DecoratedKey key);
 +    public abstract UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition);
 +    public abstract SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition);
 +    public abstract Iterator<Row> applyToPartition(Iterator<Row> partition);
 +    public abstract DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo);
 +    public abstract Row applyToRow(Row row);
 +    public abstract Row applyToStatic(Row row);
 +    public abstract RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker);
 +
 +    static class CloneToHeap extends EnsureOnHeap
 +    {
 +        protected BaseRowIterator<?> applyToPartition(BaseRowIterator partition)
 +        {
 +            return partition instanceof UnfilteredRowIterator
 +                   ? Transformation.apply((UnfilteredRowIterator) partition, this)
 +                   : Transformation.apply((RowIterator) partition, this);
 +        }
 +
 +        public DecoratedKey applyToPartitionKey(DecoratedKey key)
 +        {
 +            return new BufferDecoratedKey(key.getToken(), HeapAllocator.instance.clone(key.getKey()));
 +        }
 +
 +        public Row applyToRow(Row row)
 +        {
 +            if (row == null)
 +                return null;
 +            return Rows.copy(row, HeapAllocator.instance.cloningBTreeRowBuilder()).build();
 +        }
 +
 +        public Row applyToStatic(Row row)
 +        {
 +            if (row == Rows.EMPTY_STATIC_ROW)
 +                return row;
 +            return applyToRow(row);
 +        }
 +
 +        public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +        {
 +            return marker.copy(HeapAllocator.instance);
 +        }
 +
 +        public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition)
 +        {
 +            return Transformation.apply(partition, this);
 +        }
 +
 +        public SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition)
 +        {
 +            return new SearchIterator<Clustering, Row>()
 +            {
 +                public boolean hasNext()
 +                {
 +                    return partition.hasNext();
 +                }
 +
 +                public Row next(Clustering key)
 +                {
 +                    return applyToRow(partition.next(key));
 +                }
 +            };
 +        }
 +
 +        public Iterator<Row> applyToPartition(Iterator<Row> partition)
 +        {
 +            return new Iterator<Row>()
 +            {
 +                public boolean hasNext()
 +                {
 +                    return partition.hasNext();
 +                }
 +                public Row next()
 +                {
 +                    return applyToRow(partition.next());
 +                }
 +                public void remove()
 +                {
 +                    partition.remove();
 +                }
 +            };
 +        }
 +
 +        public DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo)
 +        {
 +            return deletionInfo.copy(HeapAllocator.instance);
 +        }
 +    }
 +
 +    static class NoOp extends EnsureOnHeap
 +    {
 +        protected BaseRowIterator<?> applyToPartition(BaseRowIterator partition)
 +        {
 +            return partition;
 +        }
 +
 +        public DecoratedKey applyToPartitionKey(DecoratedKey key)
 +        {
 +            return key;
 +        }
 +
 +        public Row applyToRow(Row row)
 +        {
 +            return row;
 +        }
 +
 +        public Row applyToStatic(Row row)
 +        {
 +            return row;
 +        }
 +
 +        public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 +        {
 +            return marker;
 +        }
 +
 +        public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition)
 +        {
 +            return partition;
 +        }
 +
 +        public SearchIterator<Clustering, Row> applyToPartition(SearchIterator<Clustering, Row> partition)
 +        {
 +            return partition;
 +        }
 +
 +        public Iterator<Row> applyToPartition(Iterator<Row> partition)
 +        {
 +            return partition;
 +        }
 +
 +        public DeletionInfo applyToDeletionInfo(DeletionInfo deletionInfo)
 +        {
 +            return deletionInfo;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/cql3/statements/PropertyDefinitionsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/statements/PropertyDefinitionsTest.java
index 417fcdc,0000000..18487f7
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/cql3/statements/PropertyDefinitionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/statements/PropertyDefinitionsTest.java
@@@ -1,61 -1,0 +1,81 @@@
++/*
++ *
++ * 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.cql3.statements;
 +
 +import org.junit.After;
 +import org.junit.Test;
 +import org.junit.Before;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +public class PropertyDefinitionsTest {
 +    
 +    PropertyDefinitions pd;
 +    
 +    @Before
 +    public void setUp()
 +    {
 +        pd = new PropertyDefinitions();
 +    }
 +    
 +    @After
 +    public void clear()
 +    {
 +        pd = null;
 +    }
 +    
 +
 +    @Test
 +    public void testGetBooleanExistant()
 +    {
 +        String key = "one";
 +        pd.addProperty(key, "1");
 +        assertEquals(Boolean.TRUE, pd.getBoolean(key, null));
 +        
 +        key = "TRUE";
 +        pd.addProperty(key, "TrUe");
 +        assertEquals(Boolean.TRUE, pd.getBoolean(key, null));
 +        
 +        key = "YES";
 +        pd.addProperty(key, "YeS");
 +        assertEquals(Boolean.TRUE, pd.getBoolean(key, null));
 +   
 +        key = "BAD_ONE";
 +        pd.addProperty(key, " 1");
 +        assertEquals(Boolean.FALSE, pd.getBoolean(key, null));
 +        
 +        key = "BAD_TRUE";
 +        pd.addProperty(key, "true ");
 +        assertEquals(Boolean.FALSE, pd.getBoolean(key, null));
 +        
 +        key = "BAD_YES";
 +        pd.addProperty(key, "ye s");
 +        assertEquals(Boolean.FALSE, pd.getBoolean(key, null));
 +    }
 +    
 +    @Test
 +    public void testGetBooleanNonexistant()
 +    {
 +        assertEquals(Boolean.FALSE, pd.getBoolean("nonexistant", Boolean.FALSE));
 +        assertEquals(Boolean.TRUE, pd.getBoolean("nonexistant", Boolean.TRUE));
 +    }
 +    
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTablesIteratedTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTablesIteratedTest.java
index 720108a,0000000..ad7bd15
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTablesIteratedTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTablesIteratedTest.java
@@@ -1,455 -1,0 +1,475 @@@
++/*
++ *
++ * 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.cql3.validation.miscellaneous;
 +
 +import org.junit.Test;
 +
 +import static org.junit.Assert.assertEquals;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.cql3.CQLTester;
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.metrics.ClearableHistogram;
 +
 +/**
 + * Tests for checking how many sstables we access during cql queries with LIMIT specified,
 + * see CASSANDRA-8180.
 + */
 +public class SSTablesIteratedTest extends CQLTester
 +{
 +    private void executeAndCheck(String query, int numSSTables, Object[]... rows) throws Throwable
 +    {
 +        ColumnFamilyStore cfs = getCurrentColumnFamilyStore();
 +
 +        ((ClearableHistogram) cfs.metric.sstablesPerReadHistogram.cf).clear(); // resets counts
 +
 +        assertRows(execute(query), rows);
 +
 +        assertEquals(numSSTables, cfs.metric.sstablesPerReadHistogram.cf.getSnapshot().getMax()); // max sstables read
 +    }
 +
 +    @Override
 +    protected String createTable(String query)
 +    {
 +        String ret = super.createTable(query);
 +        disableCompaction();
 +        return ret;
 +    }
 +
 +    @Test
 +    public void testSSTablesOnlyASC() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col ASC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 1, row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 2, row(1, 10, "10"), row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 3, row(1, 10, "10"), row(1, 20, "20"), row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 3, row(1, 10, "10"), row(1, 20, "20"), row(1, 30, "30"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 1, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 1, row(1, 10, "10"));
 +    }
 +
 +    @Test
 +    public void testMixedMemtableSStablesASC() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col ASC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 0, row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 1, row(1, 10, "10"), row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 2, row(1, 10, "10"), row(1, 20, "20"), row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 2, row(1, 10, "10"), row(1, 20, "20"), row(1, 30, "30"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 1, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 0, row(1, 10, "10"));
 +    }
 +
 +    @Test
 +    public void testOverlappingSStablesASC() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col ASC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 1, row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 2, row(1, 10, "10"), row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 2, row(1, 10, "10"), row(1, 20, "20"), row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 2, row(1, 10, "10"), row(1, 20, "20"), row(1, 30, "30"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 1, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 1, row(1, 10, "10"));
 +    }
 +
 +    @Test
 +    public void testSSTablesOnlyDESC() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 1, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 2, row(1, 30, "30"), row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 3, row(1, 30, "30"), row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 3, row(1, 30, "30"), row(1, 20, "20"), row(1, 10, "10"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 1, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 1, row(1, 30, "30"));
 +    }
 +
 +    @Test
 +    public void testMixedMemtableSStablesDESC() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 0, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 1, row(1, 30, "30"), row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 2, row(1, 30, "30"), row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 2, row(1, 30, "30"), row(1, 20, "20"), row(1, 10, "10"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 0, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 0, row(1, 30, "30"));
 +    }
 +
 +    @Test
 +    public void testOverlappingSStablesDESC() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 1, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 2, row(1, 30, "30"), row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 2, row(1, 30, "30"), row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 2, row(1, 30, "30"), row(1, 20, "20"), row(1, 10, "10"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 1, row(1, 30, "30"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 1, row(1, 30, "30"));
 +    }
 +
 +    @Test
 +    public void testDeletionOnDifferentSSTables() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        flush();
 +
 +        execute("DELETE FROM %s WHERE id=1 and col=30");
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 3, row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 4, row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 4, row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 4, row(1, 20, "20"), row(1, 10, "10"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 2);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 3, row(1, 20, "20"));
 +    }
 +
 +    @Test
 +    public void testDeletionOnSameSSTable() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        execute("DELETE FROM %s WHERE id=1 and col=30");
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 2, row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 3, row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 3, row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 3, row(1, 20, "20"), row(1, 10, "10"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 1);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 2, row(1, 20, "20"));
 +    }
 +
 +    @Test
 +    public void testDeletionOnMemTable() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        execute("DELETE FROM %s WHERE id=1 and col=30");
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 1, row(1, 20, "20"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 2, row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 3", 2, row(1, 20, "20"), row(1, 10, "10"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 2, row(1, 20, "20"), row(1, 10, "10"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 25 LIMIT 1", 0);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col < 40 LIMIT 1", 1, row(1, 20, "20"));
 +    }
 +
 +    @Test
 +    public void testDeletionOnIndexedSSTableDESC() throws Throwable
 +    {
 +        testDeletionOnIndexedSSTableDESC(true);
 +        testDeletionOnIndexedSSTableDESC(false);
 +    }
 +
 +    private void testDeletionOnIndexedSSTableDESC(boolean deleteWithRange) throws Throwable
 +    {
 +        // reduce the column index size so that columns get indexed during flush
 +        DatabaseDescriptor.setColumnIndexSize(1);
 +
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        for (int i = 1; i <= 1000; i++)
 +        {
 +            execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +        }
 +        flush();
 +
 +        Object[][] allRows = new Object[1000][];
 +        for (int i = 1001; i <= 2000; i++)
 +        {
 +            execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +            allRows[2000 - i] = row(1, i, Integer.toString(i));
 +        }
 +
 +        if (deleteWithRange)
 +        {
 +            execute("DELETE FROM %s WHERE id=1 and col <= ?", 1000);
 +        }
 +        else
 +        {
 +            for (int i = 1; i <= 1000; i++)
 +                execute("DELETE FROM %s WHERE id=1 and col = ?", i);
 +        }
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 1, row(1, 2000, "2000"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 1, row(1, 2000, "2000"), row(1, 1999, "1999"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 2, allRows);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 1000 LIMIT 1", 1, row(1, 2000, "2000"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col <= 2000 LIMIT 1", 1, row(1, 2000, "2000"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 1000", 1, allRows);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col <= 2000", 2, allRows);
 +    }
 +
 +    @Test
 +    public void testDeletionOnIndexedSSTableASC() throws Throwable
 +    {
 +        testDeletionOnIndexedSSTableASC(true);
 +        testDeletionOnIndexedSSTableASC(false);
 +    }
 +
 +    private void testDeletionOnIndexedSSTableASC(boolean deleteWithRange) throws Throwable
 +    {
 +        // reduce the column index size so that columns get indexed during flush
 +        DatabaseDescriptor.setColumnIndexSize(1);
 +
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col ASC)");
 +
 +        for (int i = 1; i <= 1000; i++)
 +        {
 +            execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +        }
 +        flush();
 +
 +        Object[][] allRows = new Object[1000][];
 +        for (int i = 1001; i <= 2000; i++)
 +        {
 +            execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +            allRows[i - 1001] = row(1, i, Integer.toString(i));
 +        }
 +        flush();
 +
 +        if (deleteWithRange)
 +        {
 +            execute("DELETE FROM %s WHERE id =1 and col <= ?", 1000);
 +        }
 +        else
 +        {
 +            for (int i = 1; i <= 1000; i++)
 +                execute("DELETE FROM %s WHERE id=1 and col = ?", i);
 +        }
 +        flush();
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 3, row(1, 1001, "1001"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 3, row(1, 1001, "1001"), row(1, 1002, "1002"));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 3, allRows);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 1000 LIMIT 1", 2, row(1, 1001, "1001"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col <= 2000 LIMIT 1", 3, row(1, 1001, "1001"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 1000", 2, allRows);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col <= 2000", 3, allRows);
 +    }
 +
 +    @Test
 +    public void testDeletionOnOverlappingIndexedSSTable() throws Throwable
 +    {
 +        testDeletionOnOverlappingIndexedSSTable(true);
 +        testDeletionOnOverlappingIndexedSSTable(false);
 +    }
 +
 +    private void testDeletionOnOverlappingIndexedSSTable(boolean deleteWithRange) throws Throwable
 +    {
 +        // reduce the column index size so that columns get indexed during flush
 +        DatabaseDescriptor.setColumnIndexSize(1);
 +
 +        createTable("CREATE TABLE %s (id int, col int, val1 text, val2 text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col ASC)");
 +
 +        for (int i = 1; i <= 500; i++)
 +        {
 +            if (i % 2 == 0)
 +                execute("INSERT INTO %s (id, col, val1) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +            else
 +                execute("INSERT INTO %s (id, col, val1, val2) VALUES (?, ?, ?, ?)", 1, i, Integer.toString(i), Integer.toString(i));
 +        }
 +
 +        for (int i = 1001; i <= 1500; i++)
 +        {
 +            if (i % 2 == 0)
 +                execute("INSERT INTO %s (id, col, val1) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +            else
 +                execute("INSERT INTO %s (id, col, val1, val2) VALUES (?, ?, ?, ?)", 1, i, Integer.toString(i), Integer.toString(i));
 +        }
 +
 +        flush();
 +
 +        for (int i = 501; i <= 1000; i++)
 +        {
 +            if (i % 2 == 0)
 +                execute("INSERT INTO %s (id, col, val1) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +            else
 +                execute("INSERT INTO %s (id, col, val1, val2) VALUES (?, ?, ?, ?)", 1, i, Integer.toString(i), Integer.toString(i));
 +        }
 +
 +        for (int i = 1501; i <= 2000; i++)
 +        {
 +            if (i % 2 == 0)
 +                execute("INSERT INTO %s (id, col, val1) VALUES (?, ?, ?)", 1, i, Integer.toString(i));
 +            else
 +                execute("INSERT INTO %s (id, col, val1, val2) VALUES (?, ?, ?, ?)", 1, i, Integer.toString(i), Integer.toString(i));
 +        }
 +
 +        if (deleteWithRange)
 +        {
 +            execute("DELETE FROM %s WHERE id=1 and col > ? and col <= ?", 250, 750);
 +        }
 +        else
 +        {
 +            for (int i = 251; i <= 750; i++)
 +                execute("DELETE FROM %s WHERE id=1 and col = ?", i);
 +        }
 +
 +        flush();
 +
 +        Object[][] allRows = new Object[1500][]; // non deleted rows
 +        for (int i = 1; i <= 2000; i++)
 +        {
 +            if (i > 250 && i <= 750)
 +                continue; // skip deleted records
 +
 +            int idx = (i <= 250 ? i - 1 : i - 501);
 +
 +            if (i % 2 == 0)
 +                allRows[idx] = row(1, i, Integer.toString(i), null);
 +            else
 +                allRows[idx] = row(1, i, Integer.toString(i), Integer.toString(i));
 +        }
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 1", 2, row(1, 1, "1", "1"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 LIMIT 2", 2, row(1, 1, "1", "1"), row(1, 2, "2", null));
 +
 +        executeAndCheck("SELECT * FROM %s WHERE id=1", 2, allRows);
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 1000 LIMIT 1", 2, row(1, 1001, "1001", "1001"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col <= 2000 LIMIT 1", 2, row(1, 1, "1", "1"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col > 500 LIMIT 1", 2, row(1, 751, "751", "751"));
 +        executeAndCheck("SELECT * FROM %s WHERE id=1 AND col <= 500 LIMIT 1", 2, row(1, 1, "1", "1"));
 +    }
 +
 +    @Test
 +    public void testMultiplePartitionsDESC() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (id int, col int, val text, PRIMARY KEY (id, col)) WITH CLUSTERING ORDER BY (col DESC)");
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 10, "10");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 2, 10, "10");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 3, 10, "10");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 20, "20");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 2, 20, "20");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 3, 20, "20");
 +        flush();
 +
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 1, 30, "30");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 2, 30, "30");
 +        execute("INSERT INTO %s (id, col, val) VALUES (?, ?, ?)", 3, 30, "30");
 +        flush();
 +
 +        for (int i = 1; i <= 3; i++)
 +        {
 +            String base = "SELECT * FROM %s ";
 +
 +            executeAndCheck(base + String.format("WHERE id=%d LIMIT 1", i), 1, row(i, 30, "30"));
 +            executeAndCheck(base + String.format("WHERE id=%d LIMIT 2", i), 2, row(i, 30, "30"), row(i, 20, "20"));
 +            executeAndCheck(base + String.format("WHERE id=%d LIMIT 3", i), 3, row(i, 30, "30"), row(i, 20, "20"), row(i, 10, "10"));
 +            executeAndCheck(base + String.format("WHERE id=%d", i), 3, row(i, 30, "30"), row(i, 20, "20"), row(i, 10, "10"));
 +
 +            executeAndCheck(base + String.format("WHERE id=%d AND col > 25 LIMIT 1", i), 1, row(i, 30, "30"));
 +            executeAndCheck(base + String.format("WHERE id=%d AND col < 40 LIMIT 1", i), 1, row(i, 30, "30"));
 +        }
 +    }
- }
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/db/marshal/AbstractCompositeTypeTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/marshal/AbstractCompositeTypeTest.java
index 0e91532,0000000..dc78bb9
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/db/marshal/AbstractCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/AbstractCompositeTypeTest.java
@@@ -1,35 -1,0 +1,55 @@@
++/*
++ *
++ * 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.db.marshal;
 +
 +import org.junit.Test;
 +import static org.junit.Assert.assertEquals;
 +
 +public class AbstractCompositeTypeTest
 +{
 +    
 +    @Test
 +    public void testEscape()
 +    {
 +        assertEquals("", AbstractCompositeType.escape(""));
 +        assertEquals("Ab!CdXy \\Z123-345", AbstractCompositeType.escape("Ab!CdXy \\Z123-345"));
 +        assertEquals("Ab!CdXy \\Z123-345!!", AbstractCompositeType.escape("Ab!CdXy \\Z123-345!"));
 +        assertEquals("Ab!CdXy \\Z123-345\\!", AbstractCompositeType.escape("Ab!CdXy \\Z123-345\\"));
 +        
 +        assertEquals("A\\:b!CdXy \\\\:Z123-345", AbstractCompositeType.escape("A:b!CdXy \\:Z123-345"));
 +        assertEquals("A\\:b!CdXy \\\\:Z123-345!!", AbstractCompositeType.escape("A:b!CdXy \\:Z123-345!"));
 +        assertEquals("A\\:b!CdXy \\\\:Z123-345\\!", AbstractCompositeType.escape("A:b!CdXy \\:Z123-345\\"));
 +        
 +    }
 +    
 +    @Test
 +    public void testUnescape()
 +    {
 +        assertEquals("", AbstractCompositeType.escape(""));
 +        assertEquals("Ab!CdXy \\Z123-345", AbstractCompositeType.unescape("Ab!CdXy \\Z123-345"));
 +        assertEquals("Ab!CdXy \\Z123-345!", AbstractCompositeType.unescape("Ab!CdXy \\Z123-345!!"));
 +        assertEquals("Ab!CdXy \\Z123-345\\", AbstractCompositeType.unescape("Ab!CdXy \\Z123-345\\!"));
 +        
 +        assertEquals("A:b!CdXy \\:Z123-345", AbstractCompositeType.unescape("A\\:b!CdXy \\\\:Z123-345"));
 +        assertEquals("A:b!CdXy \\:Z123-345!", AbstractCompositeType.unescape("A\\:b!CdXy \\\\:Z123-345!!"));
 +        assertEquals("A:b!CdXy \\:Z123-345\\", AbstractCompositeType.unescape("A\\:b!CdXy \\\\:Z123-345\\!"));
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/index/CustomIndexTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java
index 0258b8c,0000000..e18e005
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java
+++ b/test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java
@@@ -1,34 -1,0 +1,54 @@@
++/*
++ *
++ * 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.metrics;
 +
 +import org.junit.Test;
 +import org.apache.cassandra.metrics.CassandraMetricsRegistry.MetricName;
 +import static org.junit.Assert.*;
 +
 +
 +public class CassandraMetricsRegistryTest
 +{
 +    // A class with a name ending in '$'
 +    private static class StrangeName$
 +    {
 +    }
 +
 +    @Test
 +    public void testChooseType()
 +    {
 +        assertEquals("StrangeName", MetricName.chooseType(null, StrangeName$.class));
 +        assertEquals("StrangeName", MetricName.chooseType("", StrangeName$.class));
 +        assertEquals("String", MetricName.chooseType(null, String.class));
 +        assertEquals("String", MetricName.chooseType("", String.class));
 +        
 +        assertEquals("a", MetricName.chooseType("a", StrangeName$.class));
 +        assertEquals("b", MetricName.chooseType("b", String.class));
 +    }
 +    
 +    @Test
 +    public void testMetricName()
 +    {
 +         MetricName name = new MetricName(StrangeName$.class, "NaMe", "ScOpE");
 +         assertEquals("StrangeName", name.getType());
 +    }
 +    
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
index 901a5aa,0000000..785ed73
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
+++ b/test/unit/org/apache/cassandra/schema/IndexMetadataTest.java
@@@ -1,36 -1,0 +1,56 @@@
++/*
++ *
++ * 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.schema;
 +
 +import org.junit.Assert;
 +import org.junit.Test;
 +
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +
 +public class IndexMetadataTest {
 +    
 +    @Test
 +    public void testIsNameValidPositive()
 +    {
 +        assertTrue(IndexMetadata.isNameValid("abcdefghijklmnopqrstuvwxyz"));
 +        assertTrue(IndexMetadata.isNameValid("ABCDEFGHIJKLMNOPQRSTUVWXYZ"));
 +        assertTrue(IndexMetadata.isNameValid("_01234567890"));
 +    }
 +    
 +    @Test
 +    public void testIsNameValidNegative()
 +    {
 +        assertFalse(IndexMetadata.isNameValid(null));
 +        assertFalse(IndexMetadata.isNameValid(""));
 +        assertFalse(IndexMetadata.isNameValid(" "));
 +        assertFalse(IndexMetadata.isNameValid("@"));
 +        assertFalse(IndexMetadata.isNameValid("!"));
 +    }
 +    
 +    @Test
 +    public void testGetDefaultIndexName()
 +    {
 +        Assert.assertEquals("aB4__idx", IndexMetadata.getDefaultIndexName("a B-4@!_+", null));
 +        Assert.assertEquals("34_Ddd_F6_idx", IndexMetadata.getDefaultIndexName("34_()Ddd", "#F%6*"));
 +        
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/security/CipherFactoryTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/security/CipherFactoryTest.java
index 53bccc7,0000000..4ba265e
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/security/CipherFactoryTest.java
+++ b/test/unit/org/apache/cassandra/security/CipherFactoryTest.java
@@@ -1,99 -1,0 +1,119 @@@
++/*
++ *
++ * 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.security;
 +
 +import java.io.IOException;
 +import java.security.SecureRandom;
 +
 +import javax.crypto.BadPaddingException;
 +import javax.crypto.Cipher;
 +import javax.crypto.IllegalBlockSizeException;
 +
 +import com.google.common.base.Charsets;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import org.apache.cassandra.config.TransparentDataEncryptionOptions;
 +
 +public class CipherFactoryTest
 +{
 +    // http://www.gutenberg.org/files/4300/4300-h/4300-h.htm
 +    static final String ULYSSEUS = "Stately, plump Buck Mulligan came from the stairhead, bearing a bowl of lather on which a mirror and a razor lay crossed. " +
 +                                   "A yellow dressinggown, ungirdled, was sustained gently behind him on the mild morning air. He held the bowl aloft and intoned: " +
 +                                   "-Introibo ad altare Dei.";
 +    TransparentDataEncryptionOptions encryptionOptions;
 +    CipherFactory cipherFactory;
 +    SecureRandom secureRandom;
 +
 +    @Before
 +    public void setup()
 +    {
 +        secureRandom = new SecureRandom(new byte[] {0,1,2,3,4,5,6,7,8,9} );
 +        encryptionOptions = EncryptionContextGenerator.createEncryptionOptions();
 +        cipherFactory = new CipherFactory(encryptionOptions);
 +    }
 +
 +    @Test
 +    public void roundTrip() throws IOException, BadPaddingException, IllegalBlockSizeException
 +    {
 +        Cipher encryptor = cipherFactory.getEncryptor(encryptionOptions.cipher, encryptionOptions.key_alias);
 +        byte[] original = ULYSSEUS.getBytes(Charsets.UTF_8);
 +        byte[] encrypted = encryptor.doFinal(original);
 +
 +        Cipher decryptor = cipherFactory.getDecryptor(encryptionOptions.cipher, encryptionOptions.key_alias, encryptor.getIV());
 +        byte[] decrypted = decryptor.doFinal(encrypted);
 +        Assert.assertEquals(ULYSSEUS, new String(decrypted, Charsets.UTF_8));
 +    }
 +
 +    private byte[] nextIV()
 +    {
 +        byte[] b = new byte[16];
 +        secureRandom.nextBytes(b);
 +        return b;
 +    }
 +
 +    @Test
 +    public void buildCipher_SameParams() throws Exception
 +    {
 +        byte[] iv = nextIV();
 +        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.ENCRYPT_MODE);
 +        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.ENCRYPT_MODE);
 +        Assert.assertTrue(c1 == c2);
 +    }
 +
 +    @Test
 +    public void buildCipher_DifferentModes() throws Exception
 +    {
 +        byte[] iv = nextIV();
 +        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.ENCRYPT_MODE);
 +        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.DECRYPT_MODE);
 +        Assert.assertFalse(c1 == c2);
 +    }
 +
 +    @Test
 +    public void buildCipher_DifferentIVs() throws Exception
 +    {
 +        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, nextIV(), Cipher.ENCRYPT_MODE);
 +        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, nextIV(), Cipher.DECRYPT_MODE);
 +        Assert.assertFalse(c1 == c2);
 +    }
 +
 +    @Test
 +    public void buildCipher_DifferentAliases() throws Exception
 +    {
 +        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, nextIV(), Cipher.ENCRYPT_MODE);
 +        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, EncryptionContextGenerator.KEY_ALIAS_2, nextIV(), Cipher.DECRYPT_MODE);
 +        Assert.assertFalse(c1 == c2);
 +    }
 +
 +    @Test(expected = AssertionError.class)
 +    public void getDecryptor_NullIv() throws IOException
 +    {
 +        cipherFactory.getDecryptor(encryptionOptions.cipher, encryptionOptions.key_alias, null);
 +    }
 +
 +    @Test(expected = AssertionError.class)
 +    public void getDecryptor_EmptyIv() throws IOException
 +    {
 +        cipherFactory.getDecryptor(encryptionOptions.cipher, encryptionOptions.key_alias, new byte[0]);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/693e52d6/test/unit/org/apache/cassandra/service/RMIServerSocketFactoryImplTest.java
----------------------------------------------------------------------