Repository: cassandra Updated Branches: refs/heads/trunk 189124091 -> 6801cb041
Remove fat client mode Patch by brandonwilliams, reviewed by Joshua McKenzie for CASSANDRA-7820 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6801cb04 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6801cb04 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6801cb04 Branch: refs/heads/trunk Commit: 6801cb0415f40d6be1333262ae95befd8abd9175 Parents: 1891240 Author: Brandon Williams <brandonwilli...@apache.org> Authored: Tue Oct 14 10:36:04 2014 -0500 Committer: Brandon Williams <brandonwilli...@apache.org> Committed: Tue Oct 14 10:36:04 2014 -0500 ---------------------------------------------------------------------- NEWS.txt | 1 + examples/client_only/README.txt | 49 -- examples/client_only/bin/client_only | 56 -- examples/client_only/build.xml | 70 --- examples/client_only/conf/cassandra.yaml | 625 ------------------- examples/client_only/src/ClientOnlyExample.java | 114 ---- .../org/apache/cassandra/config/Config.java | 12 - .../cassandra/config/DatabaseDescriptor.java | 14 +- .../org/apache/cassandra/db/DefsTables.java | 78 +-- .../org/apache/cassandra/db/Directories.java | 5 - src/java/org/apache/cassandra/db/Keyspace.java | 3 +- src/java/org/apache/cassandra/gms/Gossiper.java | 4 +- .../hadoop/AbstractBulkRecordWriter.java | 1 - .../cassandra/io/sstable/SSTableLoader.java | 1 - .../cassandra/io/sstable/SSTableReader.java | 5 +- .../cassandra/service/AbstractReadExecutor.java | 5 - .../cassandra/service/MigrationManager.java | 2 +- .../cassandra/service/StorageService.java | 93 +-- .../locator/DynamicEndpointSnitchTest.java | 2 +- .../cassandra/service/InitClientTest.java | 33 - .../service/StorageServiceClientTest.java | 46 -- 21 files changed, 48 insertions(+), 1171 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/NEWS.txt ---------------------------------------------------------------------- diff --git a/NEWS.txt b/NEWS.txt index 99144ca..9e8f726 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -25,6 +25,7 @@ New features Upgrading --------- + - Fat client support has been removed since we have push notifications to clients - cassandra-cli has been removed. Please use cqlsh instead. - YamlFileNetworkTopologySnitch has been removed; switch to GossipingPropertyFileSnitch instead. http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/examples/client_only/README.txt ---------------------------------------------------------------------- diff --git a/examples/client_only/README.txt b/examples/client_only/README.txt deleted file mode 100644 index 47e6a3d..0000000 --- a/examples/client_only/README.txt +++ /dev/null @@ -1,49 +0,0 @@ -The client_only example uses the fat client to insert data into and read -data from Cassandra. - --- Warning -- -The method used in this example (the fat client) should generally -not be used instead of the thrift interface because of possible -instability of the internal Cassandra API. - --- Prerequisite -- -Build the Cassandra source in the current source tree. Also, if -running the client_only code against a local Cassandra node, start -the local node prior to running the client_only script. See the -configuration below for more info. - --- Build -- -To build, run ant from the contrib/client_only directory. It will build -the source, then jar up the compiled class, the conf/cassandra.yaml, and -dependencies into build/client_only.jar. - --- Run -- -To run, from the contrib/client_only directory run: -bin/client_only write -or -bin/client_only read - -'write' will create keyspace Keyspace1 and column family Standard1. If -it is already there, it will error out. It will then write a bunch of -data to the cluster it connects to. - -'read' will read the data that was written in the write step. - --- Configuration -- -The conf/cassandra.yaml is to start up the fat client. The fat client -joins the gossip network but does not participate in storage. It -needs to have the same configuration as the rest of the cluster except -listen address and rpc address. If you are running your cluster just -on your local machine, you'll need to use another address for this node. -Therefore, your local full Cassandra node can be 127.0.0.1 and the fat -client can be 127.0.0.2. Such aliasing is enabled by default on linux. -On Mac OS X, use the following command to use the second IP address: -sudo ifconfig lo0 alias 127.0.0.2 up - -cassandra.yaml can be on the classpath as is done here, can be specified -(by modifying the script) in a location within the classpath like this: -java -Xmx1G -Dcassandra.config=/path/in/classpath/to/cassandra.yaml ... -or can be retrieved from a location outside the classpath like this: -... -Dcassandra.config=file:///path/to/cassandra.yaml ... -or -... -Dcassandra.config=http://awesomesauce.com/cassandra.yaml ... \ No newline at end of file http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/examples/client_only/bin/client_only ---------------------------------------------------------------------- diff --git a/examples/client_only/bin/client_only b/examples/client_only/bin/client_only deleted file mode 100755 index af2a623..0000000 --- a/examples/client_only/bin/client_only +++ /dev/null @@ -1,56 +0,0 @@ -#!/bin/sh - -# 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. - -cwd=`dirname $0` - -name=client_only - -# Cassandra class files -if [ ! -d $cwd/../../../build/classes/main ]; then - echo "Unable to locate cassandra class files" >&2 - exit 1 -fi - -# output jar -if [ ! -e $cwd/../build/$name.jar ]; then - echo "Unable to locate $name jar" >&2 - exit 1 -fi - -CLASSPATH=$CLASSPATH:$cwd/../build/$name.jar -CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main -for jar in $cwd/../../../lib/*.jar; do - CLASSPATH=$CLASSPATH:$jar -done -for jar in $cwd/../../../build/lib/jars/*.jar; do - CLASSPATH=$CLASSPATH:$jar -done - -if [ -x $JAVA_HOME/bin/java ]; then - JAVA=$JAVA_HOME/bin/java -else - JAVA=`which java` -fi - -if [ "x$JAVA" = "x" ]; then - echo "Java executable not found (hint: set JAVA_HOME)" >&2 - exit 1 -fi - -#echo $CLASSPATH -$JAVA -Xmx1G -ea -cp $CLASSPATH ClientOnlyExample $@ http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/examples/client_only/build.xml ---------------------------------------------------------------------- diff --git a/examples/client_only/build.xml b/examples/client_only/build.xml deleted file mode 100644 index 1052d43..0000000 --- a/examples/client_only/build.xml +++ /dev/null @@ -1,70 +0,0 @@ -<?xml version="1.0" encoding="UTF-8"?> -<!-- - ~ 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. - --> -<project default="jar" name="client_only"> - <property name="cassandra.dir" value="../.." /> - <property name="cassandra.dir.lib" value="${cassandra.dir}/lib" /> - <property name="cassandra.classes" value="${cassandra.dir}/build/classes/main" /> - <property name="build.src" value="${basedir}/src" /> - <property name="build.dir" value="${basedir}/build" /> - <property name="build.classes" value="${build.dir}/classes" /> - <property name="final.name" value="client_only" /> - - <path id="build.classpath"> - <!-- cassandra dependencies --> - <fileset dir="${cassandra.dir.lib}"> - <include name="**/*.jar" /> - </fileset> - <fileset dir="${cassandra.dir}/build/lib/jars"> - <include name="**/*.jar" /> - </fileset> - <pathelement location="${cassandra.classes}" /> - </path> - - <target name="init"> - <mkdir dir="${build.classes}" /> - </target> - - <target name="build" depends="init"> - <javac destdir="${build.classes}" debug="true" - includeantruntime="false"> - <src path="${build.src}" /> - <classpath refid="build.classpath" /> - </javac> - </target> - - <target name="jar" depends="build"> - <mkdir dir="${build.classes}/META-INF" /> - <jar jarfile="${build.dir}/${final.name}.jar"> - <fileset dir="${build.classes}" /> - <fileset dir="${cassandra.classes}" /> - <fileset dir="${cassandra.dir}"> - <include name="lib/**/*.jar" /> - </fileset> - <zipfileset dir="${cassandra.dir}/build/lib/jars/" prefix="lib"> - <include name="**/*.jar" /> - </zipfileset> - <fileset file="${basedir}/conf/cassandra.yaml" /> - </jar> - </target> - - <target name="clean"> - <delete dir="${build.dir}" /> - </target> -</project> http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/examples/client_only/conf/cassandra.yaml ---------------------------------------------------------------------- diff --git a/examples/client_only/conf/cassandra.yaml b/examples/client_only/conf/cassandra.yaml deleted file mode 100644 index a6b3b43..0000000 --- a/examples/client_only/conf/cassandra.yaml +++ /dev/null @@ -1,625 +0,0 @@ -# Cassandra storage config YAML - -# NOTE: -# See http://wiki.apache.org/cassandra/StorageConfiguration for -# full explanations of configuration directives -# /NOTE - -# The name of the cluster. This is mainly used to prevent machines in -# one logical cluster from joining another. -cluster_name: 'Test Cluster' - -# This defines the number of tokens randomly assigned to this node on the ring -# The more tokens, relative to other nodes, the larger the proportion of data -# that this node will store. You probably want all nodes to have the same number -# of tokens assuming they have equal hardware capability. -# -# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, -# and will use the initial_token as described below. -# -# Specifying initial_token will override this setting. -# -# If you already have a cluster with 1 token per node, and wish to migrate to -# multiple tokens per node, see http://wiki.apache.org/cassandra/Operations -# num_tokens: 256 - -# If you haven't specified num_tokens, or have set it to the default of 1 then -# you should always specify InitialToken when setting up a production -# cluster for the first time, and often when adding capacity later. -# The principle is that each node should be given an equal slice of -# the token ring; see http://wiki.apache.org/cassandra/Operations -# for more details. -# -# If blank, Cassandra will request a token bisecting the range of -# the heaviest-loaded existing node. If there is no load information -# available, such as is the case with a new cluster, it will pick -# a random token, which will lead to hot spots. -initial_token: - -# See http://wiki.apache.org/cassandra/HintedHandoff -hinted_handoff_enabled: true -# this defines the maximum amount of time a dead host will have hints -# generated. After it has been dead this long, new hints for it will not be -# created until it has been seen alive and gone down again. -max_hint_window_in_ms: 10800000 # 3 hours -# throttle in KB's per second, per delivery thread -hinted_handoff_throttle_in_kb: 1024 -# Number of threads with which to deliver hints; -# Consider increasing this number when you have multi-dc deployments, since -# cross-dc handoff tends to be slower -max_hints_delivery_threads: 2 - -# authentication backend, implementing IAuthenticator; used to identify users -authenticator: org.apache.cassandra.auth.AllowAllAuthenticator - -# authorization backend, implementing IAuthorizer; used to limit access/provide permissions -authorizer: org.apache.cassandra.auth.AllowAllAuthorizer - -# The partitioner is responsible for distributing rows (by key) across -# nodes in the cluster. Any IPartitioner may be used, including your -# own as long as it is on the classpath. Out of the box, Cassandra -# provides org.apache.cassandra.dht.{Murmur3Partitioner, RandomPartitioner -# ByteOrderedPartitioner, OrderPreservingPartitioner (deprecated)}. -# -# - RandomPartitioner distributes rows across the cluster evenly by md5. -# This is the default prior to 1.2 and is retained for compatibility. -# - Murmur3Partitioner is similar to RandomPartioner but uses Murmur3_128 -# Hash Function instead of md5. When in doubt, this is the best option. -# - ByteOrderedPartitioner orders rows lexically by key bytes. BOP allows -# scanning rows in key order, but the ordering can generate hot spots -# for sequential insertion workloads. -# - OrderPreservingPartitioner is an obsolete form of BOP, that stores -# - keys in a less-efficient format and only works with keys that are -# UTF8-encoded Strings. -# - CollatingOPP colates according to EN,US rules rather than lexical byte -# ordering. Use this as an example if you need custom collation. -# -# See http://wiki.apache.org/cassandra/Operations for more on -# partitioners and token selection. -partitioner: org.apache.cassandra.dht.Murmur3Partitioner - -# directories where Cassandra should store data on disk. -data_file_directories: - - /tmp/fat-client/data - -# commit log -commitlog_directory: /tmp/fat-client/commitlog - -# policy for data disk failures: -# stop: shut down gossip and Thrift, leaving the node effectively dead, but -# still inspectable via JMX. -# best_effort: stop using the failed disk and respond to requests based on -# remaining available sstables. This means you WILL see obsolete -# data at CL.ONE! -# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra -disk_failure_policy: stop - -# Maximum size of the key cache in memory. -# -# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the -# minimum, sometimes more. The key cache is fairly tiny for the amount of -# time it saves, so it's worthwhile to use it at large numbers. -# The row cache saves even more time, but must store the whole values of -# its rows, so it is extremely space-intensive. It's best to only use the -# row cache if you have hot rows or static rows. -# -# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. -# -# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache. -key_cache_size_in_mb: - -# Duration in seconds after which Cassandra should -# safe the keys cache. Caches are saved to saved_caches_directory as -# specified in this configuration file. -# -# Saved caches greatly improve cold-start speeds, and is relatively cheap in -# terms of I/O for the key cache. Row cache saving is much more expensive and -# has limited use. -# -# Default is 14400 or 4 hours. -key_cache_save_period: 14400 - -# Number of keys from the key cache to save -# Disabled by default, meaning all keys are going to be saved -# key_cache_keys_to_save: 100 - -# Maximum size of the row cache in memory. -# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. -# -# Default value is 0, to disable row caching. -row_cache_size_in_mb: 0 - -# Duration in seconds after which Cassandra should -# safe the row cache. Caches are saved to saved_caches_directory as specified -# in this configuration file. -# -# Saved caches greatly improve cold-start speeds, and is relatively cheap in -# terms of I/O for the key cache. Row cache saving is much more expensive and -# has limited use. -# -# Default is 0 to disable saving the row cache. -row_cache_save_period: 0 - -# Number of keys from the row cache to save -# Disabled by default, meaning all keys are going to be saved -# row_cache_keys_to_save: 100 - -# The provider for the row cache to use. -# -# Supported values are: ConcurrentLinkedHashCacheProvider, SerializingCacheProvider -# -# SerializingCacheProvider serialises the contents of the row and stores -# it in native memory, i.e., off the JVM Heap. Serialized rows take -# significantly less memory than "live" rows in the JVM, so you can cache -# more rows in a given memory footprint. And storing the cache off-heap -# means you can use smaller heap sizes, reducing the impact of GC pauses. -# -# It is also valid to specify the fully-qualified class name to a class -# that implements org.apache.cassandra.cache.IRowCacheProvider. -# -# Defaults to SerializingCacheProvider -row_cache_provider: SerializingCacheProvider - -# The pluggable Memory allocation for Off heap row cache, Experiments show that JEMAlloc -# saves some memory than the native GCC allocator. -# -# Supported values are: NativeAllocator, JEMallocAllocator -# -# If you intend to use JEMallocAllocator you have to install JEMalloc as library and -# modify cassandra-env.sh as directed in the file. -# -# Defaults to NativeAllocator -# memory_allocator: NativeAllocator - -# saved caches -saved_caches_directory: /tmp/fat-client/saved_caches - -# commitlog_sync may be either "periodic" or "batch." -# When in batch mode, Cassandra won't ack writes until the commit log -# has been fsynced to disk. It will wait up to -# commitlog_sync_batch_window_in_ms milliseconds for other writes, before -# performing the sync. -# -# commitlog_sync: batch -# commitlog_sync_batch_window_in_ms: 50 -# -# the other option is "periodic" where writes may be acked immediately -# and the CommitLog is simply synced every commitlog_sync_period_in_ms -# milliseconds. -commitlog_sync: periodic -commitlog_sync_period_in_ms: 10000 - -# The size of the individual commitlog file segments. A commitlog -# segment may be archived, deleted, or recycled once all the data -# in it (potentally from each columnfamily in the system) has been -# flushed to sstables. -# -# The default size is 32, which is almost always fine, but if you are -# archiving commitlog segments (see commitlog_archiving.properties), -# then you probably want a finer granularity of archiving; 8 or 16 MB -# is reasonable. -commitlog_segment_size_in_mb: 32 - -# any class that implements the SeedProvider interface and has a -# constructor that takes a Map<String, String> of parameters will do. -seed_provider: - # Addresses of hosts that are deemed contact points. - # Cassandra nodes use this list of hosts to find each other and learn - # the topology of the ring. You must change this if you are running - # multiple nodes! - - class_name: org.apache.cassandra.locator.SimpleSeedProvider - parameters: - # seeds is actually a comma-delimited list of addresses. - # Ex: "<ip1>,<ip2>,<ip3>" - - seeds: "127.0.0.1" - -# For workloads with more data than can fit in memory, Cassandra's -# bottleneck will be reads that need to fetch data from -# disk. "concurrent_reads" should be set to (16 * number_of_drives) in -# order to allow the operations to enqueue low enough in the stack -# that the OS and drives can reorder them. -# -# On the other hand, since writes are almost never IO bound, the ideal -# number of "concurrent_writes" is dependent on the number of cores in -# your system; (8 * number_of_cores) is a good rule of thumb. -concurrent_reads: 32 -concurrent_writes: 32 - -# Total memory to use for memtables. Cassandra will flush the largest -# memtable when this much memory is used. -# If omitted, Cassandra will set it to 1/3 of the heap. -# memtable_total_space_in_mb: 2048 - -# Total space to use for commitlogs. Since commitlog segments are -# mmapped, and hence use up address space, the default size is 32 -# on 32-bit JVMs, and 1024 on 64-bit JVMs. -# -# If space gets above this value (it will round up to the next nearest -# segment multiple), Cassandra will flush every dirty CF in the oldest -# segment and remove it. So a small total commitlog space will tend -# to cause more flush activity on less-active columnfamilies. -# commitlog_total_space_in_mb: 4096 - -# This sets the amount of memtable flush writer threads. These will -# be blocked by disk io, and each one will hold a memtable in memory -# while blocked. If you have a large heap and many data directories, -# you can increase this value for better flush performance. -# By default this will be set to the amount of data directories defined. -#memtable_flush_writers: 1 - -# the number of full memtables to allow pending flush, that is, -# waiting for a writer thread. At a minimum, this should be set to -# the maximum number of secondary indexes created on a single CF. -memtable_flush_queue_size: 4 - -# Whether to, when doing sequential writing, fsync() at intervals in -# order to force the operating system to flush the dirty -# buffers. Enable this to avoid sudden dirty buffer flushing from -# impacting read latencies. Almost always a good idea on SSD:s; not -# necessarily on platters. -trickle_fsync: false -trickle_fsync_interval_in_kb: 10240 - -# TCP port, for commands and data -storage_port: 7000 - -# SSL port, for encrypted communication. Unused unless enabled in -# encryption_options -ssl_storage_port: 7001 - -# Address to bind to and tell other Cassandra nodes to connect to. You -# _must_ change this if you want multiple nodes to be able to -# communicate! -# -# Leaving it blank leaves it up to InetAddress.getLocalHost(). This -# will always do the Right Thing *if* the node is properly configured -# (hostname, name resolution, etc), and the Right Thing is to use the -# address associated with the hostname (it might not be). -# -# Setting this to 0.0.0.0 is always wrong. -listen_address: 127.0.0.2 - -# Address to broadcast to other Cassandra nodes -# Leaving this blank will set it to the same value as listen_address -# broadcast_address: 1.2.3.4 - - -# Whether to start the native transport server. -# Currently, only the thrift server is started by default because the native -# transport is considered beta. -# Please note that the address on which the native transport is bound is the -# same as the rpc_address. The port however is different and specified below. -start_native_transport: false -# port for the CQL native transport to listen for clients on -native_transport_port: 9042 -# The minimum and maximum threads for handling requests when the native -# transport is used. The meaning is those is similar to the one of -# rpc_min_threads and rpc_max_threads, though the default differ slightly and -# are the ones below: -# native_transport_min_threads: 16 -# native_transport_max_threads: 128 - - -# Whether to start the thrift rpc server. -start_rpc: false -# The address to bind the Thrift RPC service to -- clients connect -# here. Unlike ListenAddress above, you *can* specify 0.0.0.0 here if -# you want Thrift to listen on all interfaces. -# -# Leaving this blank has the same effect it does for ListenAddress, -# (i.e. it will be based on the configured hostname of the node). -rpc_address: localhost -# port for Thrift to listen for clients on -rpc_port: 9160 - -# enable or disable keepalive on rpc connections -rpc_keepalive: true - -# Cassandra provides three out-of-the-box options for the RPC Server: -# -# sync -> One thread per thrift connection. For a very large number of clients, memory -# will be your limiting factor. On a 64 bit JVM, 128KB is the minimum stack size -# per thread, and that will correspond to your use of virtual memory (but physical memory -# may be limited depending on use of stack space). -# -# hsha -> Stands for "half synchronous, half asynchronous." All thrift clients are handled -# asynchronously using a small number of threads that does not vary with the amount -# of thrift clients (and thus scales well to many clients). The rpc requests are still -# synchronous (one thread per active request). -# -# The default is sync because on Windows hsha is about 30% slower. On Linux, -# sync/hsha performance is about the same, with hsha of course using less memory. -# -# Alternatively, can provide your own RPC server by providing the fully-qualified class name -# of an o.a.c.t.TServerFactory that can create an instance of it. -rpc_server_type: sync - -# Uncomment rpc_min|max_thread to set request pool size limits. -# -# Regardless of your choice of RPC server (see above), the number of maximum requests in the -# RPC thread pool dictates how many concurrent requests are possible (but if you are using the sync -# RPC server, it also dictates the number of clients that can be connected at all). -# -# The default is unlimited and thus provide no protection against clients overwhelming the server. You are -# encouraged to set a maximum that makes sense for you in production, but do keep in mind that -# rpc_max_threads represents the maximum number of client requests this server may execute concurrently. -# -# rpc_min_threads: 16 -# rpc_max_threads: 2048 - -# uncomment to set socket buffer sizes on rpc connections -# rpc_send_buff_size_in_bytes: -# rpc_recv_buff_size_in_bytes: - -# uncomment to set socket buffer size for internode communication -# internode_send_buff_size_in_bytes: -# internode_recv_buff_size_in_bytes: - -# Frame size for thrift (maximum field length). -thrift_framed_transport_size_in_mb: 15 - -# The max length of a thrift message, including all fields and -# internal thrift overhead. -thrift_max_message_length_in_mb: 16 - -# Set to true to have Cassandra create a hard link to each sstable -# flushed or streamed locally in a backups/ subdirectory of the -# Keyspace data. Removing these links is the operator's -# responsibility. -incremental_backups: false - -# Whether or not to take a snapshot before each compaction. Be -# careful using this option, since Cassandra won't clean up the -# snapshots for you. Mostly useful if you're paranoid when there -# is a data format change. -snapshot_before_compaction: false - -# Whether or not a snapshot is taken of the data before keyspace truncation -# or dropping of column families. The STRONGLY advised default of true -# should be used to provide data safety. If you set this flag to false, you will -# lose data on truncation or drop. -auto_snapshot: true - -# Add column indexes to a row after its contents reach this size. -# Increase if your column values are large, or if you have a very large -# number of columns. The competing causes are, Cassandra has to -# deserialize this much of the row to read a single column, so you want -# it to be small - at least if you do many partial-row reads - but all -# the index data is read for each access, so you don't want to generate -# that wastefully either. -column_index_size_in_kb: 64 - -# Size limit for rows being compacted in memory. Larger rows will spill -# over to disk and use a slower two-pass compaction process. A message -# will be logged specifying the row key. -in_memory_compaction_limit_in_mb: 64 - -# Number of simultaneous compactions to allow, NOT including -# validation "compactions" for anti-entropy repair. Simultaneous -# compactions can help preserve read performance in a mixed read/write -# workload, by mitigating the tendency of small sstables to accumulate -# during a single long running compactions. The default is usually -# fine and if you experience problems with compaction running too -# slowly or too fast, you should look at -# compaction_throughput_mb_per_sec first. -# -# concurrent_compactors defaults to the number of cores. -# Uncomment to make compaction mono-threaded, the pre-0.8 default. -#concurrent_compactors: 1 - -# Multi-threaded compaction. When enabled, each compaction will use -# up to one thread per core, plus one thread per sstable being merged. -# This is usually only useful for SSD-based hardware: otherwise, -# your concern is usually to get compaction to do LESS i/o (see: -# compaction_throughput_mb_per_sec), not more. -multithreaded_compaction: false - -# Throttles compaction to the given total throughput across the entire -# system. The faster you insert data, the faster you need to compact in -# order to keep the sstable count down, but in general, setting this to -# 16 to 32 times the rate you are inserting data is more than sufficient. -# Setting this to 0 disables throttling. Note that this account for all types -# of compaction, including validation compaction. -compaction_throughput_mb_per_sec: 16 - -# Track cached row keys during compaction, and re-cache their new -# positions in the compacted sstable. Disable if you use really large -# key caches. -compaction_preheat_key_cache: true - -# Throttles all outbound streaming file transfers on this node to the -# given total throughput in Mbps. This is necessary because Cassandra does -# mostly sequential IO when streaming data during bootstrap or repair, which -# can lead to saturating the network connection and degrading rpc performance. -# When unset, the default is 400 Mbps or 50 MB/s. -# stream_throughput_outbound_megabits_per_sec: 400 - -# How long the coordinator should wait for read operations to complete -read_request_timeout_in_ms: 10000 -# How long the coordinator should wait for seq or index scans to complete -range_request_timeout_in_ms: 10000 -# How long the coordinator should wait for writes to complete -write_request_timeout_in_ms: 10000 -# How long the coordinator should wait for truncates to complete -# (This can be much longer, because unless auto_snapshot is disabled -# we need to flush first so we can snapshot before removing the data.) -truncate_request_timeout_in_ms: 60000 -# The default timeout for other, miscellaneous operations -request_timeout_in_ms: 10000 - -# Enable operation timeout information exchange between nodes to accurately -# measure request timeouts, If disabled cassandra will assuming the request -# was forwarded to the replica instantly by the coordinator -# -# Warning: before enabling this property make sure to ntp is installed -# and the times are synchronized between the nodes. -cross_node_timeout: false - -# Enable socket timeout for streaming operation. -# When a timeout occurs during streaming, streaming is retried from the start -# of the current file. This *can* involve re-streaming an important amount of -# data, so you should avoid setting the value too low. -# Default value is 0, which never timeout streams. -# streaming_socket_timeout_in_ms: 0 - -# phi value that must be reached for a host to be marked down. -# most users should never need to adjust this. -# phi_convict_threshold: 8 - -# endpoint_snitch -- Set this to a class that implements -# IEndpointSnitch. The snitch has two functions: -# - it teaches Cassandra enough about your network topology to route -# requests efficiently -# - it allows Cassandra to spread replicas around your cluster to avoid -# correlated failures. It does this by grouping machines into -# "datacenters" and "racks." Cassandra will do its best not to have -# more than one replica on the same "rack" (which may not actually -# be a physical location) -# -# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER, -# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS -# ARE PLACED. -# -# Out of the box, Cassandra provides -# - SimpleSnitch: -# Treats Strategy order as proximity. This improves cache locality -# when disabling read repair, which can further improve throughput. -# Only appropriate for single-datacenter deployments. -# - PropertyFileSnitch: -# Proximity is determined by rack and data center, which are -# explicitly configured in cassandra-topology.properties. -# - GossipingPropertyFileSnitch -# The rack and datacenter for the local node are defined in -# cassandra-rackdc.properties and propagated to other nodes via gossip. If -# cassandra-topology.properties exists, it is used as a fallback, allowing -# migration from the PropertyFileSnitch. -# - RackInferringSnitch: -# Proximity is determined by rack and data center, which are -# assumed to correspond to the 3rd and 2nd octet of each node's -# IP address, respectively. Unless this happens to match your -# deployment conventions (as it did Facebook's), this is best used -# as an example of writing a custom Snitch class. -# - Ec2Snitch: -# Appropriate for EC2 deployments in a single Region. Loads Region -# and Availability Zone information from the EC2 API. The Region is -# treated as the Datacenter, and the Availability Zone as the rack. -# Only private IPs are used, so this will not work across multiple -# Regions. -# - Ec2MultiRegionSnitch: -# Uses public IPs as broadcast_address to allow cross-region -# connectivity. (Thus, you should set seed addresses to the public -# IP as well.) You will need to open the storage_port or -# ssl_storage_port on the public IP firewall. (For intra-Region -# traffic, Cassandra will switch to the private IP after -# establishing a connection.) -# -# You can use a custom Snitch by setting this to the full class name -# of the snitch, which will be assumed to be on your classpath. -endpoint_snitch: SimpleSnitch - -# controls how often to perform the more expensive part of host score -# calculation -dynamic_snitch_update_interval_in_ms: 100 -# controls how often to reset all host scores, allowing a bad host to -# possibly recover -dynamic_snitch_reset_interval_in_ms: 600000 -# if set greater than zero and read_repair_chance is < 1.0, this will allow -# 'pinning' of replicas to hosts in order to increase cache capacity. -# The badness threshold will control how much worse the pinned host has to be -# before the dynamic snitch will prefer other replicas over it. This is -# expressed as a double which represents a percentage. Thus, a value of -# 0.2 means Cassandra would continue to prefer the static snitch values -# until the pinned host was 20% worse than the fastest. -dynamic_snitch_badness_threshold: 0.1 - -# request_scheduler -- Set this to a class that implements -# RequestScheduler, which will schedule incoming client requests -# according to the specific policy. This is useful for multi-tenancy -# with a single Cassandra cluster. -# NOTE: This is specifically for requests from the client and does -# not affect inter node communication. -# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place -# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of -# client requests to a node with a separate queue for each -# request_scheduler_id. The scheduler is further customized by -# request_scheduler_options as described below. -request_scheduler: org.apache.cassandra.scheduler.NoScheduler - -# Scheduler Options vary based on the type of scheduler -# NoScheduler - Has no options -# RoundRobin -# - throttle_limit -- The throttle_limit is the number of in-flight -# requests per client. Requests beyond -# that limit are queued up until -# running requests can complete. -# The value of 80 here is twice the number of -# concurrent_reads + concurrent_writes. -# - default_weight -- default_weight is optional and allows for -# overriding the default which is 1. -# - weights -- Weights are optional and will default to 1 or the -# overridden default_weight. The weight translates into how -# many requests are handled during each turn of the -# RoundRobin, based on the scheduler id. -# -# request_scheduler_options: -# throttle_limit: 80 -# default_weight: 5 -# weights: -# Keyspace1: 1 -# Keyspace2: 5 - -# request_scheduler_id -- An identifer based on which to perform -# the request scheduling. Currently the only valid option is keyspace. -# request_scheduler_id: keyspace - -# Enable or disable inter-node encryption -# Default settings are TLS v1, RSA 1024-bit keys (it is imperative that -# users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher -# suite for authentication, key exchange and encryption of the actual data transfers. -# NOTE: No custom encryption options are enabled at the moment -# The available internode options are : all, none, dc, rack -# -# If set to dc cassandra will encrypt the traffic between the DCs -# If set to rack cassandra will encrypt the traffic between the racks -# -# The passwords used in these options must match the passwords used when generating -# the keystore and truststore. For instructions on generating these files, see: -# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore -# -server_encryption_options: - internode_encryption: none - keystore: conf/.keystore - keystore_password: cassandra - truststore: conf/.truststore - truststore_password: cassandra - # More advanced defaults below: - # protocol: TLS - # algorithm: SunX509 - # store_type: JKS - # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA] - # require_client_auth: false - -# enable or disable client/server encryption. -client_encryption_options: - enabled: false - keystore: conf/.keystore - keystore_password: cassandra - # More advanced defaults below: - # protocol: TLS - # algorithm: SunX509 - # store_type: JKS - # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA] - # require_client_auth: false - -# internode_compression controls whether traffic between nodes is -# compressed. -# can be: all - all traffic is compressed -# dc - traffic between different datacenters is compressed -# none - nothing is compressed. -internode_compression: all - -# Enable or disable tcp_nodelay for inter-dc communication. -# Disabling it will result in larger (but fewer) network packets being sent, -# reducing overhead from the TCP protocol itself, at the cost of increasing -# latency if you block for cross-datacenter responses. -inter_dc_tcp_nodelay: false http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/examples/client_only/src/ClientOnlyExample.java ---------------------------------------------------------------------- diff --git a/examples/client_only/src/ClientOnlyExample.java b/examples/client_only/src/ClientOnlyExample.java deleted file mode 100644 index 7b42140..0000000 --- a/examples/client_only/src/ClientOnlyExample.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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. - */ - -import java.util.concurrent.TimeUnit; - -import com.google.common.util.concurrent.Uninterruptibles; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.cql3.QueryProcessor; -import org.apache.cassandra.cql3.UntypedResultSet; -import org.apache.cassandra.db.ConsistencyLevel; -import org.apache.cassandra.exceptions.RequestExecutionException; -import org.apache.cassandra.exceptions.RequestValidationException; -import org.apache.cassandra.service.*; - -public class ClientOnlyExample -{ - private static final Logger logger = LoggerFactory.getLogger(ClientOnlyExample.class); - - private static final String KEYSPACE = "keyspace1"; - private static final String COLUMN_FAMILY = "standard1"; - - private static void startClient() throws Exception - { - StorageService.instance.initClient(); - } - - private static void testWriting() throws Exception - { - // do some writing. - for (int i = 0; i < 100; i++) - { - QueryProcessor.process(String.format("INSERT INTO %s.%s (id, name, value) VALUES ( 'key%d', 'colb', 'value%d')", - KEYSPACE, - COLUMN_FAMILY, - i, - i), - ConsistencyLevel.QUORUM); - - System.out.println("wrote key" + i); - } - System.out.println("Done writing."); - } - - private static void testReading() throws Exception - { - // do some queries. - for (int i = 0; i < 100; i++) - { - String query = String.format("SELECT id, name, value FROM %s.%s WHERE id = 'key%d'", - KEYSPACE, - COLUMN_FAMILY, - i); - UntypedResultSet.Row row = QueryProcessor.process(query, ConsistencyLevel.QUORUM).one(); - System.out.println(String.format("ID: %s, Name: %s, Value: %s", row.getString("id"), row.getString("name"), row.getString("value"))); - } - } - - /** - * First, bring one or more nodes up. Then run ClientOnlyExample with these VM arguments: - * <p/> - * -Xmx1G - * -Dcassandra.config=/Users/gary/cassandra/conf/cassandra.yaml (optional, will first look for cassandra.yaml on classpath) - * <p/> - * Pass "write" or "read" into the program to exercise the various methods. - * <p/> - * Caveats: - * <p/> - * 1. Much of cassandra is not reentrant. That is, you can't spin a client up, down, then back up in the same jvm. - * 2. Because of the above, you still need to force-quit the process. StorageService.stopClient() doesn't (can't) - * spin everything down. - */ - public static void main(String args[]) throws Exception - { - startClient(); - setupKeyspace(); - testWriting(); - logger.info("Writing is done. Sleeping, then will try to read."); - Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); - - testReading(); - - // no need to do this: - // StorageService.instance().decommission(); - // do this instead: - StorageService.instance.stopClient(); - System.exit(0); // the only way to really stop the process. - } - - private static void setupKeyspace() throws RequestExecutionException, RequestValidationException, InterruptedException - { - QueryProcessor.process("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", - ConsistencyLevel.ANY); - QueryProcessor.process("CREATE TABLE IF NOT EXISTS " + KEYSPACE + "." + COLUMN_FAMILY + " (id ascii PRIMARY KEY, name ascii, value ascii )", - ConsistencyLevel.ANY); - TimeUnit.MILLISECONDS.sleep(1000); - } -} http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/config/Config.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 0f525cd..bb992e6 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -197,8 +197,6 @@ public class Config public String memory_allocator = NativeAllocator.class.getSimpleName(); - private static boolean isClientMode = false; - public Integer file_cache_size_in_mb; public boolean inter_dc_tcp_nodelay = true; @@ -226,16 +224,6 @@ public class Config outboundBindAny = value; } - public static boolean isClientMode() - { - return isClientMode; - } - - public static void setClientMode(boolean clientMode) - { - isClientMode = clientMode; - } - public void configHintedHandoff() throws ConfigurationException { if (hinted_handoff_enabled != null && !hinted_handoff_enabled.isEmpty()) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/config/DatabaseDescriptor.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 64d7ec8..319801d 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -115,21 +115,9 @@ public class DatabaseDescriptor static { - // In client mode, we use a default configuration. Note that the fields of this class will be - // left unconfigured however (the partitioner or localDC will be null for instance) so this - // should be used with care. try { - if (Config.isClientMode()) - { - conf = new Config(); - // at least we have to set memoryAllocator to open SSTable in client mode - memoryAllocator = FBUtilities.newOffHeapAllocator(conf.memory_allocator); - } - else - { - applyConfig(loadConfig()); - } + applyConfig(loadConfig()); } catch (ConfigurationException e) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/db/DefsTables.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java index 1693914..748d3f4 100644 --- a/src/java/org/apache/cassandra/db/DefsTables.java +++ b/src/java/org/apache/cassandra/db/DefsTables.java @@ -123,7 +123,7 @@ public class DefsTables for (Mutation mutation : mutations) mutation.apply(); - if (doFlush && !StorageService.instance.isClientMode()) + if (doFlush) flushSchemaCFs(); // with new data applied @@ -353,11 +353,8 @@ public class DefsTables assert Schema.instance.getKSMetaData(ksm.name) == null; Schema.instance.load(ksm); - if (!StorageService.instance.isClientMode()) - { - Keyspace.open(ksm.name); - MigrationManager.instance.notifyCreateKeyspace(ksm); - } + Keyspace.open(ksm.name); + MigrationManager.instance.notifyCreateKeyspace(ksm); } private static void addColumnFamily(CFMetaData cfm) @@ -375,12 +372,8 @@ public class DefsTables Keyspace.open(cfm.ksName); Schema.instance.setKeyspaceDefinition(ksm); - - if (!StorageService.instance.isClientMode()) - { - Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true); - MigrationManager.instance.notifyCreateColumnFamily(cfm); - } + Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true); + MigrationManager.instance.notifyCreateColumnFamily(cfm); } private static void addType(UserType ut) @@ -392,8 +385,7 @@ public class DefsTables ksm.userTypes.addType(ut); - if (!StorageService.instance.isClientMode()) - MigrationManager.instance.notifyCreateUserType(ut); + MigrationManager.instance.notifyCreateUserType(ut); } private static void addFunction(UDFunction udf) @@ -402,8 +394,7 @@ public class DefsTables Functions.addFunction(udf); - if (!StorageService.instance.isClientMode()) - MigrationManager.instance.notifyCreateFunction(udf); + MigrationManager.instance.notifyCreateFunction(udf); } private static void updateKeyspace(String ksName) @@ -414,11 +405,8 @@ public class DefsTables Schema.instance.setKeyspaceDefinition(newKsm); - if (!StorageService.instance.isClientMode()) - { - Keyspace.open(ksName).createReplicationStrategy(newKsm); - MigrationManager.instance.notifyUpdateKeyspace(newKsm); - } + Keyspace.open(ksName).createReplicationStrategy(newKsm); + MigrationManager.instance.notifyUpdateKeyspace(newKsm); } private static void updateColumnFamily(String ksName, String cfName) @@ -427,12 +415,9 @@ public class DefsTables assert cfm != null; cfm.reload(); - if (!StorageService.instance.isClientMode()) - { - Keyspace keyspace = Keyspace.open(cfm.ksName); - keyspace.getColumnFamilyStore(cfm.cfName).reload(); - MigrationManager.instance.notifyUpdateColumnFamily(cfm); - } + Keyspace keyspace = Keyspace.open(cfm.ksName); + keyspace.getColumnFamilyStore(cfm.cfName).reload(); + MigrationManager.instance.notifyUpdateColumnFamily(cfm); } private static void updateType(UserType ut) @@ -444,8 +429,7 @@ public class DefsTables ksm.userTypes.addType(ut); - if (!StorageService.instance.isClientMode()) - MigrationManager.instance.notifyUpdateUserType(ut); + MigrationManager.instance.notifyUpdateUserType(ut); } private static void updateFunction(UDFunction udf) @@ -454,8 +438,7 @@ public class DefsTables Functions.replaceFunction(udf); - if (!StorageService.instance.isClientMode()) - MigrationManager.instance.notifyUpdateFunction(udf); + MigrationManager.instance.notifyUpdateFunction(udf); } private static void dropKeyspace(String ksName) @@ -475,12 +458,9 @@ public class DefsTables Schema.instance.purge(cfm); - if (!StorageService.instance.isClientMode()) - { - if (DatabaseDescriptor.isAutoSnapshot()) - cfs.snapshot(snapshotName); - Keyspace.open(ksm.name).dropCf(cfm.cfId); - } + if (DatabaseDescriptor.isAutoSnapshot()) + cfs.snapshot(snapshotName); + Keyspace.open(ksm.name).dropCf(cfm.cfId); droppedCfs.add(cfm.cfId); } @@ -494,10 +474,7 @@ public class DefsTables // force a new segment in the CL CommitLog.instance.forceRecycleAllSegments(droppedCfs); - if (!StorageService.instance.isClientMode()) - { - MigrationManager.instance.notifyDropKeyspace(ksm); - } + MigrationManager.instance.notifyDropKeyspace(ksm); } private static void dropColumnFamily(String ksName, String cfName) @@ -515,15 +492,12 @@ public class DefsTables CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true); - if (!StorageService.instance.isClientMode()) - { - if (DatabaseDescriptor.isAutoSnapshot()) - cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name)); - Keyspace.open(ksm.name).dropCf(cfm.cfId); - MigrationManager.instance.notifyDropColumnFamily(cfm); + if (DatabaseDescriptor.isAutoSnapshot()) + cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name)); + Keyspace.open(ksm.name).dropCf(cfm.cfId); + MigrationManager.instance.notifyDropColumnFamily(cfm); - CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId)); - } + CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId)); } private static void dropType(UserType ut) @@ -533,8 +507,7 @@ public class DefsTables ksm.userTypes.removeType(ut); - if (!StorageService.instance.isClientMode()) - MigrationManager.instance.notifyDropUserType(ut); + MigrationManager.instance.notifyDropUserType(ut); } private static void dropFunction(UDFunction udf) @@ -544,8 +517,7 @@ public class DefsTables // TODO: this is kind of broken as this remove all overloads of the function name Functions.removeFunction(udf.name(), udf.argTypes()); - if (!StorageService.instance.isClientMode()) - MigrationManager.instance.notifyDropFunction(udf); + MigrationManager.instance.notifyDropFunction(udf); } private static KSMetaData makeNewKeyspaceDefinition(KSMetaData ksm, CFMetaData toExclude) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/db/Directories.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java index d62ebeb..bd3281a 100644 --- a/src/java/org/apache/cassandra/db/Directories.java +++ b/src/java/org/apache/cassandra/db/Directories.java @@ -182,11 +182,6 @@ public class Directories public Directories(CFMetaData metadata) { this.metadata = metadata; - if (StorageService.instance.isClientMode()) - { - dataPaths = null; - return; - } String cfId = ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(metadata.cfId)); int idx = metadata.cfName.indexOf(SECONDARY_INDEX_NAME_SEPARATOR); http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/db/Keyspace.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index 86f1487..ca43df6 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -68,8 +68,7 @@ public class Keyspace // proper directories here as well as in CassandraDaemon. static { - if (!StorageService.instance.isClientMode()) - DatabaseDescriptor.createAllDirectories(); + DatabaseDescriptor.createAllDirectories(); } public final KSMetaData metadata; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/gms/Gossiper.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 9e29e40..c488b09 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -628,7 +628,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean } } - public boolean isFatClient(InetAddress endpoint) + public boolean isGossipOnlyMember(InetAddress endpoint) { EndpointState epState = endpointStateMap.get(endpoint); if (epState == null) @@ -672,7 +672,7 @@ public class Gossiper implements IFailureDetectionEventListener, GossiperMBean { // check if this is a fat client. fat clients are removed automatically from // gossip after FatClientTimeout. Do not remove dead states here. - if (isFatClient(endpoint) + if (isGossipOnlyMember(endpoint) && !justRemovedEndpoints.containsKey(endpoint) && TimeUnit.NANOSECONDS.toMillis(nowNano - epState.getUpdateTimestamp()) > FatClientTimeout) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java index 22255a6..9ec37f4 100644 --- a/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java +++ b/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java @@ -85,7 +85,6 @@ implements org.apache.hadoop.mapred.RecordWriter<K, V> protected AbstractBulkRecordWriter(Configuration conf) { - Config.setClientMode(true); Config.setOutboundBindAny(true); this.conf = conf; DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(Integer.parseInt(conf.get(STREAM_THROTTLE_MBITS, "0"))); http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java index 991fa1d..68bc6ab 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java @@ -57,7 +57,6 @@ public class SSTableLoader implements StreamEventHandler static { - Config.setClientMode(true); } public SSTableLoader(File directory, Client client, OutputHandler outputHandler) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/io/sstable/SSTableReader.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java index b897496..1bc7a24 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java @@ -506,10 +506,9 @@ public class SSTableReader extends SSTable deletingTask = new SSTableDeletingTask(this); - // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist - // the read meter when in client mode. Also don't track reads for special operations (like early open) + // Don't track read rates for tables in the system keyspace. Also don't track reads for special operations (like early open) // this is to avoid overflowing the executor queue (see CASSANDRA-8066) - if (Keyspace.SYSTEM_KS.equals(desc.ksname) || Config.isClientMode() || openReason != OpenReason.NORMAL) + if (Keyspace.SYSTEM_KS.equals(desc.ksname) || openReason != OpenReason.NORMAL) { readMeter = null; readMeterSyncFuture = null; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/service/AbstractReadExecutor.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java index d08c63e..061a01b 100644 --- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java +++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java @@ -157,11 +157,6 @@ public abstract class AbstractReadExecutor // Throw UAE early if we don't have enough replicas. consistencyLevel.assureSufficientLiveNodes(keyspace, targetReplicas); - // Fat client. Speculating read executors need access to cfs metrics and sampled latency, and fat clients - // can't provide that. So, for now, fat clients will always use NeverSpeculatingReadExecutor. - if (StorageService.instance.isClientMode()) - return new NeverSpeculatingReadExecutor(command, consistencyLevel, targetReplicas); - if (repairDecision != ReadRepairDecision.NONE) ReadRepairMetrics.attempted.mark(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/service/MigrationManager.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java index 4f03483..449560c 100644 --- a/src/java/org/apache/cassandra/service/MigrationManager.java +++ b/src/java/org/apache/cassandra/service/MigrationManager.java @@ -148,7 +148,7 @@ public class MigrationManager */ return MessagingService.instance().knowsVersion(endpoint) && MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version - && !Gossiper.instance.isFatClient(endpoint); + && !Gossiper.instance.isGossipOnlyMember(endpoint); } public static boolean isReadyForBootstrap() http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index eb4c3e2..0982047 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -178,15 +178,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE /* we bootstrap but do NOT join the ring unless told to do so */ private boolean isSurveyMode= Boolean.parseBoolean(System.getProperty("cassandra.write_survey", "false")); - /* when intialized as a client, we shouldn't write to the system keyspace. */ - private boolean isClientMode; private boolean initialized; private volatile boolean joined = false; /* the probability for tracing any particular request, 0 disables tracing and 1 enables for all */ private double traceProbability = 0.0; - private static enum Mode { STARTING, NORMAL, CLIENT, JOINING, LEAVING, DECOMMISSIONED, MOVING, DRAINING, DRAINED } + private static enum Mode { STARTING, NORMAL, JOINING, LEAVING, DECOMMISSIONED, MOVING, DRAINING, DRAINED } private Mode operationMode = Mode.STARTING; /* Used for tracking drain progress */ @@ -455,7 +453,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE MessagingService.instance().listen(FBUtilities.getLocalAddress()); Gossiper.instance.doShadowRound(); EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(FBUtilities.getBroadcastAddress()); - if (epState != null && !Gossiper.instance.isDeadState(epState) && !Gossiper.instance.isFatClient(FBUtilities.getBroadcastAddress())) + if (epState != null && !Gossiper.instance.isDeadState(epState) && !Gossiper.instance.isGossipOnlyMember(FBUtilities.getBroadcastAddress())) { throw new RuntimeException(String.format("A node with address %s already exists, cancelling join. " + "Use cassandra.replace_address if you want to replace this node.", @@ -478,49 +476,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE Gossiper.instance.resetEndpointStateMap(); } - public synchronized void initClient() throws ConfigurationException + // for testing only + public void unsafeInitialize() throws ConfigurationException { - // We don't wait, because we're going to actually try to work on - initClient(0); - - // sleep a while to allow gossip to warm up (the other nodes need to know about this one before they can reply). - outer: - while (true) - { - Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); - for (InetAddress address : Gossiper.instance.getLiveMembers()) - { - if (!Gossiper.instance.isFatClient(address)) - break outer; - } - } - - // sleep until any schema migrations have finished - while (!MigrationManager.isReadyForBootstrap()) - { - Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); - } - } - - public synchronized void initClient(int ringDelay) throws ConfigurationException - { - if (initialized) - { - if (!isClientMode) - throw new UnsupportedOperationException("StorageService does not support switching modes."); - return; - } initialized = true; - isClientMode = true; - logger.info("Starting up client gossip"); - setMode(Mode.CLIENT, false); Gossiper.instance.register(this); Gossiper.instance.start((int) (System.currentTimeMillis() / 1000)); // needed for node-ring gathering. Gossiper.instance.addLocalApplicationState(ApplicationState.NET_VERSION, valueFactory.networkVersion()); - if (!MessagingService.instance().isListening()) MessagingService.instance().listen(FBUtilities.getLocalAddress()); - Uninterruptibles.sleepUninterruptibly(ringDelay, TimeUnit.MILLISECONDS); } public synchronized void initServer() throws ConfigurationException @@ -534,14 +498,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE logger.info("Thrift API version: {}", cassandraConstants.VERSION); logger.info("CQL supported versions: {} (default: {})", StringUtils.join(ClientState.getCQLSupportedVersion(), ","), ClientState.DEFAULT_CQL_VERSION); - if (initialized) - { - if (isClientMode) - throw new UnsupportedOperationException("StorageService does not support switching modes."); - return; - } initialized = true; - isClientMode = false; try { @@ -1608,21 +1565,18 @@ public class StorageService extends NotificationBroadcasterSupport implements IE { logger.debug("New node {} at token {}", endpoint, token); tokensToUpdateInMetadata.add(token); - if (!isClientMode) - tokensToUpdateInSystemKeyspace.add(token); + tokensToUpdateInSystemKeyspace.add(token); } else if (endpoint.equals(currentOwner)) { // set state back to normal, since the node may have tried to leave, but failed and is now back up tokensToUpdateInMetadata.add(token); - if (!isClientMode) - tokensToUpdateInSystemKeyspace.add(token); + tokensToUpdateInSystemKeyspace.add(token); } else if (Gossiper.instance.compareEndpointStartup(endpoint, currentOwner) > 0) { tokensToUpdateInMetadata.add(token); - if (!isClientMode) - tokensToUpdateInSystemKeyspace.add(token); + tokensToUpdateInSystemKeyspace.add(token); // currentOwner is no longer current, endpoint is. Keep track of these moves, because when // a host no longer has any tokens, we'll want to remove it. @@ -1658,12 +1612,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE if (tokenMetadata.isMoving(endpoint)) // if endpoint was moving to a new token { tokenMetadata.removeFromMoving(endpoint); - - if (!isClientMode) - { - for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers) - subscriber.onMove(endpoint); - } + for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers) + subscriber.onMove(endpoint); } PendingRangeCalculatorService.instance.update(); @@ -1803,11 +1753,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE tokenMetadata.removeEndpoint(endpoint); tokenMetadata.removeBootstrapTokens(tokens); - if (!isClientMode) - { - for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers) - subscriber.onLeaveCluster(endpoint); - } + for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers) + subscriber.onLeaveCluster(endpoint); PendingRangeCalculatorService.instance.update(); } @@ -1821,8 +1768,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE private void removeEndpoint(InetAddress endpoint) { Gossiper.instance.removeEndpoint(endpoint); - if (!isClientMode) - SystemKeyspace.removeEndpoint(endpoint); + SystemKeyspace.removeEndpoint(endpoint); } protected void addExpireTimeIfFound(InetAddress endpoint, long expireTime) @@ -2019,9 +1965,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE { MigrationManager.instance.scheduleSchemaPull(endpoint, state); - if (isClientMode) - return; - if (tokenMetadata.isMember(endpoint)) { HintedHandOffManager.instance.scheduleHintDelivery(endpoint); @@ -2044,11 +1987,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE public void onDead(InetAddress endpoint, EndpointState state) { MessagingService.instance().convict(endpoint); - if (!isClientMode) - { - for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers) - subscriber.onDown(endpoint); - } + for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers) + subscriber.onDown(endpoint); } public void onRestart(InetAddress endpoint, EndpointState state) @@ -3579,11 +3519,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE } } - public boolean isClientMode() - { - return isClientMode; - } - public synchronized void requestGC() { if (hasUnreclaimedSpace()) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java index e23bcfa..c7c1f17 100644 --- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java @@ -49,7 +49,7 @@ public class DynamicEndpointSnitchTest public void testSnitch() throws InterruptedException, IOException, ConfigurationException { // do this because SS needs to be initialized before DES can work properly. - StorageService.instance.initClient(0); + StorageService.instance.unsafeInitialize(); SimpleSnitch ss = new SimpleSnitch(); DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode())); InetAddress self = FBUtilities.getBroadcastAddress(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/test/unit/org/apache/cassandra/service/InitClientTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/service/InitClientTest.java b/test/unit/org/apache/cassandra/service/InitClientTest.java deleted file mode 100644 index 4ce0678..0000000 --- a/test/unit/org/apache/cassandra/service/InitClientTest.java +++ /dev/null @@ -1,33 +0,0 @@ -package org.apache.cassandra.service; - -import org.junit.Test; - -import org.apache.cassandra.exceptions.ConfigurationException; - -/** - * 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. - */ - - -public class InitClientTest // extends CleanupHelper -{ - @Test - public void testInitClientStartup() throws ConfigurationException - { - StorageService.instance.initClient(0); - } -} http://git-wip-us.apache.org/repos/asf/cassandra/blob/6801cb04/test/unit/org/apache/cassandra/service/StorageServiceClientTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/service/StorageServiceClientTest.java b/test/unit/org/apache/cassandra/service/StorageServiceClientTest.java deleted file mode 100644 index f3ba754..0000000 --- a/test/unit/org/apache/cassandra/service/StorageServiceClientTest.java +++ /dev/null @@ -1,46 +0,0 @@ -/* -* 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.service; - -import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.junit.Test; -import static org.junit.Assert.assertFalse; - -import java.io.File; - -public class StorageServiceClientTest -{ - @Test - public void testClientOnlyMode() throws ConfigurationException - { - SchemaLoader.mkdirs(); - SchemaLoader.cleanup(); - StorageService.instance.initClient(0); - - // verify that no storage directories were created. - for (String path : DatabaseDescriptor.getAllDataFileLocations()) - { - assertFalse(new File(path).exists()); - } - StorageService.instance.stopClient(); - } -}