Merge branch 'cassandra-3.9' into trunk
diff --git a/CHANGES.txt b/CHANGES.txt
index e7d9066..36a21e6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,27 @@
+3.10
+ * Add schema to snapshot manifest, add USING TIMESTAMP clause to ALTER TABLE statements (CASSANDRA-7190)
+ * Add beta protocol flag for v5 native protocol (CASSANDRA-12142)
+ * Support filtering on non-PRIMARY KEY columns in the CREATE
+ MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368)
+ * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004)
+ * COPY FROM should raise error for non-existing input files (CASSANDRA-12174)
+ * Faster write path (CASSANDRA-12269)
+ * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
+ * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
+ * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
+ * Prepend snapshot name with "truncated" or "dropped" when a snapshot
+ is taken before truncating or dropping a table (CASSANDRA-12178)
+ * Optimize RestrictionSet (CASSANDRA-12153)
+ * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
+ * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
+ * Create a system table to expose prepared statements (CASSANDRA-8831)
+ * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
+ * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
+ * Add supplied username to authentication error messages (CASSANDRA-12076)
+ * Remove pre-startup check for open JMX port (CASSANDRA-12074)
+ * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738)
+
+
3.9
* cqlsh: Fix handling of $$-escaped strings (CASSANDRA-12189)
* Fix SSL JMX requiring truststore containing server cert (CASSANDRA-12109)
@@ -21,7 +45,6 @@
* cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
* Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
-
3.8
* Fix hdr logging for single operation workloads (CASSANDRA-12145)
* Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)
@@ -154,7 +177,7 @@
* Add support to rebuild from specific range (CASSANDRA-10406)
* Optimize the overlapping lookup by calculating all the
bounds in advance (CASSANDRA-11571)
- * Support json/yaml output in noetool tablestats (CASSANDRA-5977)
+ * Support json/yaml output in nodetool tablestats (CASSANDRA-5977)
* (stress) Add datacenter option to -node options (CASSANDRA-11591)
* Fix handling of empty slices (CASSANDRA-11513)
* Make number of cores used by cqlsh COPY visible to testing code (CASSANDRA-11437)
diff --git a/NEWS.txt b/NEWS.txt
index 7418f3a..8580c7c 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -14,6 +14,33 @@
using the provided 'sstableupgrade' tool.
+3.10
+====
+
+New features
+------------
+ - Prepared statements are now persisted in the table prepared_statements in
+ the system keyspace. Upon startup, this table is used to preload all
+ previously prepared statements - i.e. in many cases clients do not need to
+ re-prepare statements against restarted nodes.
+ - cqlsh can now connect to older Cassandra versions by downgrading the native
+ protocol version. Please note that this is currently not part of our release
+ testing and, as a consequence, it is not guaranteed to work in all cases.
+ See CASSANDRA-12150 for more details.
+ - Snapshots that are automatically taken before a table is dropped or truncated
+ will have a "dropped" or "truncated" prefix on their snapshot tag name.
+ - Metrics are exposed for successful and failed authentication attempts.
+ These can be located using the object names org.apache.cassandra.metrics:type=Client,name=AuthSuccess
+ and org.apache.cassandra.metrics:type=Client,name=AuthFailure respectively.
+ - Add support to "unset" JSON fields in prepared statements by specifying DEFAULT UNSET.
+ See CASSANDRA-11424 for details
+
+Upgrading
+---------
+ - Logs written to stdout are now consistent with logs written to files.
+ Time is now local (it was UTC on the console and local in files). Date, thread, file
+ and line info where added to stdout. (see CASSANDRA-12004)
+
3.8
===
@@ -2145,4 +2172,3 @@
compaction deserializes each row before merging.
See https://issues.apache.org/jira/browse/CASSANDRA-16
-
diff --git a/bin/cassandra b/bin/cassandra
index 3206fdc..acf87cb 100755
--- a/bin/cassandra
+++ b/bin/cassandra
@@ -273,13 +273,6 @@
fi
fi
-# see CASSANDRA-7254
-"$JAVA" -cp "$CLASSPATH" $JVM_OPTS 2>&1 | grep -q 'Error: Exception thrown by the agent : java.lang.NullPointerException'
-if [ $? -ne "1" ]; then
- echo Unable to bind JMX, is Cassandra already running?
- exit 1;
-fi
-
# Start up the service
launch_service "$pidfile" "$foreground" "$properties" "$classname"
diff --git a/bin/cqlsh.py b/bin/cqlsh.py
index ce85449..ee23b29 100644
--- a/bin/cqlsh.py
+++ b/bin/cqlsh.py
@@ -177,7 +177,6 @@
DEFAULT_HOST = '127.0.0.1'
DEFAULT_PORT = 9042
DEFAULT_SSL = False
-DEFAULT_CQLVER = '3.4.2'
DEFAULT_PROTOCOL_VERSION = 4
DEFAULT_CONNECT_TIMEOUT_SECONDS = 5
DEFAULT_REQUEST_TIMEOUT_SECONDS = 10
@@ -219,8 +218,9 @@
parser.add_option("--encoding", help="Specify a non-default encoding for output." +
" (Default: %s)" % (UTF8,))
parser.add_option("--cqlshrc", help="Specify an alternative cqlshrc file location.")
-parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
- help='Specify a particular CQL version (default: %default).'
+parser.add_option('--cqlversion', default=None,
+ help='Specify a particular CQL version, '
+ 'by default the highest version supported by the server will be used.'
' Examples: "3.0.3", "3.1.0"')
parser.add_option("-e", "--execute", help='Execute the statement and quit.')
parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, dest='connect_timeout',
@@ -662,7 +662,7 @@
def __init__(self, hostname, port, color=False,
username=None, password=None, encoding=None, stdin=None, tty=True,
completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
- cqlver=DEFAULT_CQLVER, keyspace=None,
+ cqlver=None, keyspace=None,
tracing_enabled=False, expand_enabled=False,
display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
@@ -701,7 +701,6 @@
control_connection_timeout=connect_timeout,
connect_timeout=connect_timeout)
self.owns_connection = not use_conn
- self.set_expanded_cql_version(cqlver)
if keyspace:
self.session = self.conn.connect(keyspace)
@@ -726,6 +725,7 @@
self.session.row_factory = ordered_dict_factory
self.session.default_consistency_level = cassandra.ConsistencyLevel.ONE
self.get_connection_versions()
+ self.set_expanded_cql_version(self.connection_versions['cql'])
self.current_keyspace = keyspace
@@ -2448,7 +2448,7 @@
optvalues.encoding = option_with_default(configs.get, 'ui', 'encoding', UTF8)
optvalues.tty = option_with_default(configs.getboolean, 'ui', 'tty', sys.stdin.isatty())
- optvalues.cqlversion = option_with_default(configs.get, 'cql', 'version', DEFAULT_CQLVER)
+ optvalues.cqlversion = option_with_default(configs.get, 'cql', 'version', None)
optvalues.connect_timeout = option_with_default(configs.getint, 'connection', 'timeout', DEFAULT_CONNECT_TIMEOUT_SECONDS)
optvalues.request_timeout = option_with_default(configs.getint, 'connection', 'request_timeout', DEFAULT_REQUEST_TIMEOUT_SECONDS)
optvalues.execute = None
@@ -2492,11 +2492,11 @@
else:
options.color = should_use_color()
- options.cqlversion, cqlvertup = full_cql_version(options.cqlversion)
- if cqlvertup[0] < 3:
- parser.error('%r is not a supported CQL version.' % options.cqlversion)
- else:
- options.cqlmodule = cql3handling
+ if options.cqlversion is not None:
+ options.cqlversion, cqlvertup = full_cql_version(options.cqlversion)
+ if cqlvertup[0] < 3:
+ parser.error('%r is not a supported CQL version.' % options.cqlversion)
+ options.cqlmodule = cql3handling
try:
port = int(port)
diff --git a/build.xml b/build.xml
index e13fdc0..b8b470c 100644
--- a/build.xml
+++ b/build.xml
@@ -25,7 +25,7 @@
<property name="debuglevel" value="source,lines,vars"/>
<!-- default version and SCM information -->
- <property name="base.version" value="3.9"/>
+ <property name="base.version" value="3.10"/>
<property name="scm.connection" value="scm:git://git.apache.org/cassandra.git"/>
<property name="scm.developerConnection" value="scm:git://git.apache.org/cassandra.git"/>
<property name="scm.url" value="http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree"/>
@@ -189,6 +189,7 @@
<mkdir dir="${build.classes.thrift}"/>
<mkdir dir="${test.lib}"/>
<mkdir dir="${test.classes}"/>
+ <mkdir dir="${stress.test.classes}"/>
<mkdir dir="${build.src.gen-java}"/>
<mkdir dir="${build.dir.lib}"/>
<mkdir dir="${jacoco.export.dir}"/>
@@ -830,12 +831,29 @@
<!-- Stress build file -->
<property name="stress.build.src" value="${basedir}/tools/stress/src" />
+ <property name="stress.test.src" value="${basedir}/tools/stress/test/unit" />
<property name="stress.build.classes" value="${build.classes}/stress" />
+ <property name="stress.test.classes" value="${build.dir}/test/stress-classes" />
<property name="stress.manifest" value="${stress.build.classes}/MANIFEST.MF" />
<path id="cassandra.classes">
<pathelement location="${basedir}/build/classes/main" />
<pathelement location="${basedir}/build/classes/thrift" />
</path>
+
+ <target name="stress-build-test" depends="stress-build" description="Compile stress tests">
+ <javac debug="true" debuglevel="${debuglevel}" destdir="${stress.test.classes}"
+ includeantruntime="false"
+ source="${source.version}"
+ target="${target.version}"
+ encoding="utf-8">
+ <classpath>
+ <path refid="cassandra.classpath"/>
+ <pathelement location="${stress.build.classes}" />
+ </classpath>
+ <src path="${stress.test.src}"/>
+ </javac>
+ </target>
+
<target name="stress-build" depends="build" description="build stress tool">
<mkdir dir="${stress.build.classes}" />
<javac compiler="modern" debug="true" debuglevel="${debuglevel}" encoding="utf-8" destdir="${stress.build.classes}" includeantruntime="true" source="${source.version}" target="${target.version}">
@@ -854,6 +872,12 @@
</copy>
</target>
+ <target name="stress-test" depends="stress-build-test, build-test" description="Runs stress tests">
+ <testmacro inputdir="${stress.test.src}"
+ timeout="${test.timeout}">
+ </testmacro>
+ </target>
+
<target name="_write-poms" depends="maven-declare-dependencies">
<artifact:writepom pomRefId="parent-pom" file="${build.dir}/${final.name}-parent.pom"/>
<artifact:writepom pomRefId="thrift-pom"
@@ -1248,8 +1272,10 @@
<optjvmargs/>
<classpath>
<pathelement path="${java.class.path}"/>
+ <pathelement location="${stress.build.classes}"/>
<path refid="cassandra.classpath" />
<pathelement location="${test.classes}"/>
+ <pathelement location="${stress.test.classes}"/>
<pathelement location="${test.conf}"/>
<fileset dir="${test.lib}">
<include name="**/*.jar" />
@@ -1551,7 +1577,7 @@
</target>
<target name="test-all"
- depends="eclipse-warnings,test,long-test,test-compression,test-clientutil-jar"
+ depends="eclipse-warnings,test,long-test,test-compression,test-clientutil-jar,stress-test"
description="Run all tests except for those under test-burn" />
<!-- Use JaCoCo ant extension without needing externally saved lib -->
@@ -1818,11 +1844,13 @@
<classpathentry kind="src" path="src/java"/>
<classpathentry kind="src" path="src/resources"/>
<classpathentry kind="src" path="src/gen-java"/>
+ <classpathentry kind="src" path="conf" including="hotspot_compiler"/>
<classpathentry kind="src" path="interface/thrift/gen-java"/>
<classpathentry kind="src" output="build/test/classes" path="test/unit"/>
<classpathentry kind="src" output="build/test/classes" path="test/long"/>
<classpathentry kind="src" output="build/test/classes" path="test/resources" />
<classpathentry kind="src" path="tools/stress/src"/>
+ <classpathentry kind="src" output="build/test/stress-classes" path="tools/stress/test/unit" />
<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER"/>
<classpathentry kind="output" path="build/classes/main"/>
<classpathentry kind="lib" path="build/classes/thrift" sourcepath="interface/thrift/gen-java/"/>
diff --git a/conf/jvm.options b/conf/jvm.options
index 692d06b..9e13e0e 100644
--- a/conf/jvm.options
+++ b/conf/jvm.options
@@ -118,6 +118,7 @@
# resize them at runtime.
-XX:+UseTLAB
-XX:+ResizeTLAB
+-XX:+UseNUMA
# http://www.evanjones.ca/jvm-mmap-pause.html
-XX:+PerfDisableSharedMem
diff --git a/conf/logback-tools.xml b/conf/logback-tools.xml
index e47985e..12b59fb 100644
--- a/conf/logback-tools.xml
+++ b/conf/logback-tools.xml
@@ -21,7 +21,7 @@
<appender name="STDERR" class="ch.qos.logback.core.ConsoleAppender">
<target>System.err</target>
<encoder>
- <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
+ <pattern>%-5level %date{"HH:mm:ss,SSS"} %msg%n</pattern>
</encoder>
<filter class="ch.qos.logback.classic.filter.ThresholdFilter">
<level>WARN</level>
diff --git a/conf/logback.xml b/conf/logback.xml
index a47740d..b0c95f7 100644
--- a/conf/logback.xml
+++ b/conf/logback.xml
@@ -79,7 +79,7 @@
<level>INFO</level>
</filter>
<encoder>
- <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
+ <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
</encoder>
</appender>
diff --git a/doc/convert_yaml_to_rst.py b/doc/convert_yaml_to_rst.py
index fee6d8c..c17bbbb 100644
--- a/doc/convert_yaml_to_rst.py
+++ b/doc/convert_yaml_to_rst.py
@@ -58,6 +58,8 @@
lines = f.readlines()[7:]
with open(dest_file, 'w') as outfile:
+ outfile.write(".. _cassandra-yaml:\n")
+ outfile.write("\n")
outfile.write("Cassandra Configuration File\n")
outfile.write("============================\n")
diff --git a/doc/native_protocol_v5.spec b/doc/native_protocol_v5.spec
new file mode 100644
index 0000000..edf3093
--- /dev/null
+++ b/doc/native_protocol_v5.spec
@@ -0,0 +1,1162 @@
+
+ CQL BINARY PROTOCOL v5
+
+
+Table of Contents
+
+ 1. Overview
+ 2. Frame header
+ 2.1. version
+ 2.2. flags
+ 2.3. stream
+ 2.4. opcode
+ 2.5. length
+ 3. Notations
+ 4. Messages
+ 4.1. Requests
+ 4.1.1. STARTUP
+ 4.1.2. AUTH_RESPONSE
+ 4.1.3. OPTIONS
+ 4.1.4. QUERY
+ 4.1.5. PREPARE
+ 4.1.6. EXECUTE
+ 4.1.7. BATCH
+ 4.1.8. REGISTER
+ 4.2. Responses
+ 4.2.1. ERROR
+ 4.2.2. READY
+ 4.2.3. AUTHENTICATE
+ 4.2.4. SUPPORTED
+ 4.2.5. RESULT
+ 4.2.5.1. Void
+ 4.2.5.2. Rows
+ 4.2.5.3. Set_keyspace
+ 4.2.5.4. Prepared
+ 4.2.5.5. Schema_change
+ 4.2.6. EVENT
+ 4.2.7. AUTH_CHALLENGE
+ 4.2.8. AUTH_SUCCESS
+ 5. Compression
+ 6. Data Type Serialization Formats
+ 7. User Defined Type Serialization
+ 8. Result paging
+ 9. Error codes
+ 10. Changes from v4
+
+
+1. Overview
+
+ The CQL binary protocol is a frame based protocol. Frames are defined as:
+
+ 0 8 16 24 32 40
+ +---------+---------+---------+---------+---------+
+ | version | flags | stream | opcode |
+ +---------+---------+---------+---------+---------+
+ | length |
+ +---------+---------+---------+---------+
+ | |
+ . ... body ... .
+ . .
+ . .
+ +----------------------------------------
+
+ The protocol is big-endian (network byte order).
+
+ Each frame contains a fixed size header (9 bytes) followed by a variable size
+ body. The header is described in Section 2. The content of the body depends
+ on the header opcode value (the body can in particular be empty for some
+ opcode values). The list of allowed opcodes is defined in Section 2.4 and the
+ details of each corresponding message are described Section 4.
+
+ The protocol distinguishes two types of frames: requests and responses. Requests
+ are those frames sent by the client to the server. Responses are those frames sent
+ by the server to the client. Note, however, that the protocol supports server pushes
+ (events) so a response does not necessarily come right after a client request.
+
+ Note to client implementors: client libraries should always assume that the
+ body of a given frame may contain more data than what is described in this
+ document. It will however always be safe to ignore the remainder of the frame
+ body in such cases. The reason is that this may enable extending the protocol
+ with optional features without needing to change the protocol version.
+
+
+
+2. Frame header
+
+2.1. version
+
+ The version is a single byte that indicates both the direction of the message
+ (request or response) and the version of the protocol in use. The most
+ significant bit of version is used to define the direction of the message:
+ 0 indicates a request, 1 indicates a response. This can be useful for protocol
+ analyzers to distinguish the nature of the packet from the direction in which
+ it is moving. The rest of that byte is the protocol version (5 for the protocol
+ defined in this document). In other words, for this version of the protocol,
+ version will be one of:
+ 0x04 Request frame for this protocol version
+ 0x84 Response frame for this protocol version
+
+ Please note that while every message ships with the version, only one version
+ of messages is accepted on a given connection. In other words, the first message
+ exchanged (STARTUP) sets the version for the connection for the lifetime of this
+ connection. The single exception to this behavior is when a startup message
+ is sent with a version that is higher than the current server version. In this
+ case, the server will respond with its current version.
+
+ This document describes version 5 of the protocol. For the changes made since
+ version 4, see Section 10.
+
+
+2.2. flags
+
+ Flags applying to this frame. The flags have the following meaning (described
+ by the mask that allows selecting them):
+ 0x01: Compression flag. If set, the frame body is compressed. The actual
+ compression to use should have been set up beforehand through the
+ Startup message (which thus cannot be compressed; Section 4.1.1).
+ 0x02: Tracing flag. For a request frame, this indicates the client requires
+ tracing of the request. Note that only QUERY, PREPARE and EXECUTE queries
+ support tracing. Other requests will simply ignore the tracing flag if
+ set. If a request supports tracing and the tracing flag is set, the response
+ to this request will have the tracing flag set and contain tracing
+ information.
+ If a response frame has the tracing flag set, its body contains
+ a tracing ID. The tracing ID is a [uuid] and is the first thing in
+ the frame body. The rest of the body will then be the usual body
+ corresponding to the response opcode.
+ 0x04: Custom payload flag. For a request or response frame, this indicates
+ that a generic key-value custom payload for a custom QueryHandler
+ implementation is present in the frame. Such a custom payload is simply
+ ignored by the default QueryHandler implementation.
+ Currently, only QUERY, PREPARE, EXECUTE and BATCH requests support
+ payload.
+ Type of custom payload is [bytes map] (see below).
+ 0x08: Warning flag. The response contains warnings which were generated by the
+ server to go along with this response.
+ If a response frame has the warning flag set, its body will contain the
+ text of the warnings. The warnings are a [string list] and will be the
+ first value in the frame body if the tracing flag is not set, or directly
+ after the tracing ID if it is.
+ 0x10: Use beta flag. Indicates that the client opts in to use protocol version
+ that is currently in beta. Server will respond with ERROR if protocol
+ version is marked as beta on server and client does not provide this flag.
+
+ The rest of flags is currently unused and ignored.
+
+2.3. stream
+
+ A frame has a stream id (a [short] value). When sending request messages, this
+ stream id must be set by the client to a non-negative value (negative stream id
+ are reserved for streams initiated by the server; currently all EVENT messages
+ (section 4.2.6) have a streamId of -1). If a client sends a request message
+ with the stream id X, it is guaranteed that the stream id of the response to
+ that message will be X.
+
+ This helps to enable the asynchronous nature of the protocol. If a client
+ sends multiple messages simultaneously (without waiting for responses), there
+ is no guarantee on the order of the responses. For instance, if the client
+ writes REQ_1, REQ_2, REQ_3 on the wire (in that order), the server might
+ respond to REQ_3 (or REQ_2) first. Assigning different stream ids to these 3
+ requests allows the client to distinguish to which request a received answer
+ responds to. As there can only be 32768 different simultaneous streams, it is up
+ to the client to reuse stream id.
+
+ Note that clients are free to use the protocol synchronously (i.e. wait for
+ the response to REQ_N before sending REQ_N+1). In that case, the stream id
+ can be safely set to 0. Clients should also feel free to use only a subset of
+ the 32768 maximum possible stream ids if it is simpler for its implementation.
+
+2.4. opcode
+
+ An integer byte that distinguishes the actual message:
+ 0x00 ERROR
+ 0x01 STARTUP
+ 0x02 READY
+ 0x03 AUTHENTICATE
+ 0x05 OPTIONS
+ 0x06 SUPPORTED
+ 0x07 QUERY
+ 0x08 RESULT
+ 0x09 PREPARE
+ 0x0A EXECUTE
+ 0x0B REGISTER
+ 0x0C EVENT
+ 0x0D BATCH
+ 0x0E AUTH_CHALLENGE
+ 0x0F AUTH_RESPONSE
+ 0x10 AUTH_SUCCESS
+
+ Messages are described in Section 4.
+
+ (Note that there is no 0x04 message in this version of the protocol)
+
+
+2.5. length
+
+ A 4 byte integer representing the length of the body of the frame (note:
+ currently a frame is limited to 256MB in length).
+
+
+3. Notations
+
+ To describe the layout of the frame body for the messages in Section 4, we
+ define the following:
+
+ [int] A 4 bytes integer
+ [long] A 8 bytes integer
+ [short] A 2 bytes unsigned integer
+ [string] A [short] n, followed by n bytes representing an UTF-8
+ string.
+ [long string] An [int] n, followed by n bytes representing an UTF-8 string.
+ [uuid] A 16 bytes long uuid.
+ [string list] A [short] n, followed by n [string].
+ [bytes] A [int] n, followed by n bytes if n >= 0. If n < 0,
+ no byte should follow and the value represented is `null`.
+ [value] A [int] n, followed by n bytes if n >= 0.
+ If n == -1 no byte should follow and the value represented is `null`.
+ If n == -2 no byte should follow and the value represented is
+ `not set` not resulting in any change to the existing value.
+ n < -2 is an invalid value and results in an error.
+ [short bytes] A [short] n, followed by n bytes if n >= 0.
+
+ [option] A pair of <id><value> where <id> is a [short] representing
+ the option id and <value> depends on that option (and can be
+ of size 0). The supported id (and the corresponding <value>)
+ will be described when this is used.
+ [option list] A [short] n, followed by n [option].
+ [inet] An address (ip and port) to a node. It consists of one
+ [byte] n, that represents the address size, followed by n
+ [byte] representing the IP address (in practice n can only be
+ either 4 (IPv4) or 16 (IPv6)), following by one [int]
+ representing the port.
+ [consistency] A consistency level specification. This is a [short]
+ representing a consistency level with the following
+ correspondance:
+ 0x0000 ANY
+ 0x0001 ONE
+ 0x0002 TWO
+ 0x0003 THREE
+ 0x0004 QUORUM
+ 0x0005 ALL
+ 0x0006 LOCAL_QUORUM
+ 0x0007 EACH_QUORUM
+ 0x0008 SERIAL
+ 0x0009 LOCAL_SERIAL
+ 0x000A LOCAL_ONE
+
+ [string map] A [short] n, followed by n pair <k><v> where <k> and <v>
+ are [string].
+ [string multimap] A [short] n, followed by n pair <k><v> where <k> is a
+ [string] and <v> is a [string list].
+ [bytes map] A [short] n, followed by n pair <k><v> where <k> is a
+ [string] and <v> is a [bytes].
+
+
+4. Messages
+
+4.1. Requests
+
+ Note that outside of their normal responses (described below), all requests
+ can get an ERROR message (Section 4.2.1) as response.
+
+4.1.1. STARTUP
+
+ Initialize the connection. The server will respond by either a READY message
+ (in which case the connection is ready for queries) or an AUTHENTICATE message
+ (in which case credentials will need to be provided using AUTH_RESPONSE).
+
+ This must be the first message of the connection, except for OPTIONS that can
+ be sent before to find out the options supported by the server. Once the
+ connection has been initialized, a client should not send any more STARTUP
+ messages.
+
+ The body is a [string map] of options. Possible options are:
+ - "CQL_VERSION": the version of CQL to use. This option is mandatory and
+ currently the only version supported is "3.0.0". Note that this is
+ different from the protocol version.
+ - "COMPRESSION": the compression algorithm to use for frames (See section 5).
+ This is optional; if not specified no compression will be used.
+
+
+4.1.2. AUTH_RESPONSE
+
+ Answers a server authentication challenge.
+
+ Authentication in the protocol is SASL based. The server sends authentication
+ challenges (a bytes token) to which the client answers with this message. Those
+ exchanges continue until the server accepts the authentication by sending a
+ AUTH_SUCCESS message after a client AUTH_RESPONSE. Note that the exchange
+ begins with the client sending an initial AUTH_RESPONSE in response to a
+ server AUTHENTICATE request.
+
+ The body of this message is a single [bytes] token. The details of what this
+ token contains (and when it can be null/empty, if ever) depends on the actual
+ authenticator used.
+
+ The response to a AUTH_RESPONSE is either a follow-up AUTH_CHALLENGE message,
+ an AUTH_SUCCESS message or an ERROR message.
+
+
+4.1.3. OPTIONS
+
+ Asks the server to return which STARTUP options are supported. The body of an
+ OPTIONS message should be empty and the server will respond with a SUPPORTED
+ message.
+
+
+4.1.4. QUERY
+
+ Performs a CQL query. The body of the message must be:
+ <query><query_parameters>
+ where <query> is a [long string] representing the query and
+ <query_parameters> must be
+ <consistency><flags>[<n>[name_1]<value_1>...[name_n]<value_n>][<result_page_size>][<paging_state>][<serial_consistency>][<timestamp>]
+ where:
+ - <consistency> is the [consistency] level for the operation.
+ - <flags> is a [byte] whose bits define the options for this query and
+ in particular influence what the remainder of the message contains.
+ A flag is set if the bit corresponding to its `mask` is set. Supported
+ flags are, given their mask:
+ 0x01: Values. If set, a [short] <n> followed by <n> [value]
+ values are provided. Those values are used for bound variables in
+ the query. Optionally, if the 0x40 flag is present, each value
+ will be preceded by a [string] name, representing the name of
+ the marker the value must be bound to.
+ 0x02: Skip_metadata. If set, the Result Set returned as a response
+ to the query (if any) will have the NO_METADATA flag (see
+ Section 4.2.5.2).
+ 0x04: Page_size. If set, <result_page_size> is an [int]
+ controlling the desired page size of the result (in CQL3 rows).
+ See the section on paging (Section 8) for more details.
+ 0x08: With_paging_state. If set, <paging_state> should be present.
+ <paging_state> is a [bytes] value that should have been returned
+ in a result set (Section 4.2.5.2). The query will be
+ executed but starting from a given paging state. This is also to
+ continue paging on a different node than the one where it
+ started (See Section 8 for more details).
+ 0x10: With serial consistency. If set, <serial_consistency> should be
+ present. <serial_consistency> is the [consistency] level for the
+ serial phase of conditional updates. That consitency can only be
+ either SERIAL or LOCAL_SERIAL and if not present, it defaults to
+ SERIAL. This option will be ignored for anything else other than a
+ conditional update/insert.
+ 0x20: With default timestamp. If set, <timestamp> should be present.
+ <timestamp> is a [long] representing the default timestamp for the query
+ in microseconds (negative values are forbidden). This will
+ replace the server side assigned timestamp as default timestamp.
+ Note that a timestamp in the query itself will still override
+ this timestamp. This is entirely optional.
+ 0x40: With names for values. This only makes sense if the 0x01 flag is set and
+ is ignored otherwise. If present, the values from the 0x01 flag will
+ be preceded by a name (see above). Note that this is only useful for
+ QUERY requests where named bind markers are used; for EXECUTE statements,
+ since the names for the expected values was returned during preparation,
+ a client can always provide values in the right order without any names
+ and using this flag, while supported, is almost surely inefficient.
+
+ Note that the consistency is ignored by some queries (USE, CREATE, ALTER,
+ TRUNCATE, ...).
+
+ The server will respond to a QUERY message with a RESULT message, the content
+ of which depends on the query.
+
+
+4.1.5. PREPARE
+
+ Prepare a query for later execution (through EXECUTE). The body consists of
+ the CQL query to prepare as a [long string].
+
+ The server will respond with a RESULT message with a `prepared` kind (0x0004,
+ see Section 4.2.5).
+
+
+4.1.6. EXECUTE
+
+ Executes a prepared query. The body of the message must be:
+ <id><query_parameters>
+ where <id> is the prepared query ID. It's the [short bytes] returned as a
+ response to a PREPARE message. As for <query_parameters>, it has the exact
+ same definition as in QUERY (see Section 4.1.4).
+
+ The response from the server will be a RESULT message.
+
+
+4.1.7. BATCH
+
+ Allows executing a list of queries (prepared or not) as a batch (note that
+ only DML statements are accepted in a batch). The body of the message must
+ be:
+ <type><n><query_1>...<query_n><consistency><flags>[<serial_consistency>][<timestamp>]
+ where:
+ - <type> is a [byte] indicating the type of batch to use:
+ - If <type> == 0, the batch will be "logged". This is equivalent to a
+ normal CQL3 batch statement.
+ - If <type> == 1, the batch will be "unlogged".
+ - If <type> == 2, the batch will be a "counter" batch (and non-counter
+ statements will be rejected).
+ - <flags> is a [byte] whose bits define the options for this query and
+ in particular influence what the remainder of the message contains. It is similar
+ to the <flags> from QUERY and EXECUTE methods, except that the 4 rightmost
+ bits must always be 0 as their corresponding options do not make sense for
+ Batch. A flag is set if the bit corresponding to its `mask` is set. Supported
+ flags are, given their mask:
+ 0x10: With serial consistency. If set, <serial_consistency> should be
+ present. <serial_consistency> is the [consistency] level for the
+ serial phase of conditional updates. That consistency can only be
+ either SERIAL or LOCAL_SERIAL and if not present, it defaults to
+ SERIAL. This option will be ignored for anything else other than a
+ conditional update/insert.
+ 0x20: With default timestamp. If set, <timestamp> should be present.
+ <timestamp> is a [long] representing the default timestamp for the query
+ in microseconds. This will replace the server side assigned
+ timestamp as default timestamp. Note that a timestamp in the query itself
+ will still override this timestamp. This is entirely optional.
+ 0x40: With names for values. If set, then all values for all <query_i> must be
+ preceded by a [string] <name_i> that have the same meaning as in QUERY
+ requests [IMPORTANT NOTE: this feature does not work and should not be
+ used. It is specified in a way that makes it impossible for the server
+ to implement. This will be fixed in a future version of the native
+ protocol. See https://issues.apache.org/jira/browse/CASSANDRA-10246 for
+ more details].
+ - <n> is a [short] indicating the number of following queries.
+ - <query_1>...<query_n> are the queries to execute. A <query_i> must be of the
+ form:
+ <kind><string_or_id><n>[<name_1>]<value_1>...[<name_n>]<value_n>
+ where:
+ - <kind> is a [byte] indicating whether the following query is a prepared
+ one or not. <kind> value must be either 0 or 1.
+ - <string_or_id> depends on the value of <kind>. If <kind> == 0, it should be
+ a [long string] query string (as in QUERY, the query string might contain
+ bind markers). Otherwise (that is, if <kind> == 1), it should be a
+ [short bytes] representing a prepared query ID.
+ - <n> is a [short] indicating the number (possibly 0) of following values.
+ - <name_i> is the optional name of the following <value_i>. It must be present
+ if and only if the 0x40 flag is provided for the batch.
+ - <value_i> is the [value] to use for bound variable i (of bound variable <name_i>
+ if the 0x40 flag is used).
+ - <consistency> is the [consistency] level for the operation.
+ - <serial_consistency> is only present if the 0x10 flag is set. In that case,
+ <serial_consistency> is the [consistency] level for the serial phase of
+ conditional updates. That consitency can only be either SERIAL or
+ LOCAL_SERIAL and if not present will defaults to SERIAL. This option will
+ be ignored for anything else other than a conditional update/insert.
+
+ The server will respond with a RESULT message.
+
+
+4.1.8. REGISTER
+
+ Register this connection to receive some types of events. The body of the
+ message is a [string list] representing the event types to register for. See
+ section 4.2.6 for the list of valid event types.
+
+ The response to a REGISTER message will be a READY message.
+
+ Please note that if a client driver maintains multiple connections to a
+ Cassandra node and/or connections to multiple nodes, it is advised to
+ dedicate a handful of connections to receive events, but to *not* register
+ for events on all connections, as this would only result in receiving
+ multiple times the same event messages, wasting bandwidth.
+
+
+4.2. Responses
+
+ This section describes the content of the frame body for the different
+ responses. Please note that to make room for future evolution, clients should
+ support extra informations (that they should simply discard) to the one
+ described in this document at the end of the frame body.
+
+4.2.1. ERROR
+
+ Indicates an error processing a request. The body of the message will be an
+ error code ([int]) followed by a [string] error message. Then, depending on
+ the exception, more content may follow. The error codes are defined in
+ Section 9, along with their additional content if any.
+
+
+4.2.2. READY
+
+ Indicates that the server is ready to process queries. This message will be
+ sent by the server either after a STARTUP message if no authentication is
+ required (if authentication is required, the server indicates readiness by
+ sending a AUTH_RESPONSE message).
+
+ The body of a READY message is empty.
+
+
+4.2.3. AUTHENTICATE
+
+ Indicates that the server requires authentication, and which authentication
+ mechanism to use.
+
+ The authentication is SASL based and thus consists of a number of server
+ challenges (AUTH_CHALLENGE, Section 4.2.7) followed by client responses
+ (AUTH_RESPONSE, Section 4.1.2). The initial exchange is however boostrapped
+ by an initial client response. The details of that exchange (including how
+ many challenge-response pairs are required) are specific to the authenticator
+ in use. The exchange ends when the server sends an AUTH_SUCCESS message or
+ an ERROR message.
+
+ This message will be sent following a STARTUP message if authentication is
+ required and must be answered by a AUTH_RESPONSE message from the client.
+
+ The body consists of a single [string] indicating the full class name of the
+ IAuthenticator in use.
+
+
+4.2.4. SUPPORTED
+
+ Indicates which startup options are supported by the server. This message
+ comes as a response to an OPTIONS message.
+
+ The body of a SUPPORTED message is a [string multimap]. This multimap gives
+ for each of the supported STARTUP options, the list of supported values.
+
+
+4.2.5. RESULT
+
+ The result to a query (QUERY, PREPARE, EXECUTE or BATCH messages).
+
+ The first element of the body of a RESULT message is an [int] representing the
+ `kind` of result. The rest of the body depends on the kind. The kind can be
+ one of:
+ 0x0001 Void: for results carrying no information.
+ 0x0002 Rows: for results to select queries, returning a set of rows.
+ 0x0003 Set_keyspace: the result to a `use` query.
+ 0x0004 Prepared: result to a PREPARE message.
+ 0x0005 Schema_change: the result to a schema altering query.
+
+ The body for each kind (after the [int] kind) is defined below.
+
+
+4.2.5.1. Void
+
+ The rest of the body for a Void result is empty. It indicates that a query was
+ successful without providing more information.
+
+
+4.2.5.2. Rows
+
+ Indicates a set of rows. The rest of the body of a Rows result is:
+ <metadata><rows_count><rows_content>
+ where:
+ - <metadata> is composed of:
+ <flags><columns_count>[<paging_state>][<global_table_spec>?<col_spec_1>...<col_spec_n>]
+ where:
+ - <flags> is an [int]. The bits of <flags> provides information on the
+ formatting of the remaining information. A flag is set if the bit
+ corresponding to its `mask` is set. Supported flags are, given their
+ mask:
+ 0x0001 Global_tables_spec: if set, only one table spec (keyspace
+ and table name) is provided as <global_table_spec>. If not
+ set, <global_table_spec> is not present.
+ 0x0002 Has_more_pages: indicates whether this is not the last
+ page of results and more should be retrieved. If set, the
+ <paging_state> will be present. The <paging_state> is a
+ [bytes] value that should be used in QUERY/EXECUTE to
+ continue paging and retrieve the remainder of the result for
+ this query (See Section 8 for more details).
+ 0x0004 No_metadata: if set, the <metadata> is only composed of
+ these <flags>, the <column_count> and optionally the
+ <paging_state> (depending on the Has_more_pages flag) but
+ no other information (so no <global_table_spec> nor <col_spec_i>).
+ This will only ever be the case if this was requested
+ during the query (see QUERY and RESULT messages).
+ - <columns_count> is an [int] representing the number of columns selected
+ by the query that produced this result. It defines the number of <col_spec_i>
+ elements in and the number of elements for each row in <rows_content>.
+ - <global_table_spec> is present if the Global_tables_spec is set in
+ <flags>. It is composed of two [string] representing the
+ (unique) keyspace name and table name the columns belong to.
+ - <col_spec_i> specifies the columns returned in the query. There are
+ <column_count> such column specifications that are composed of:
+ (<ksname><tablename>)?<name><type>
+ The initial <ksname> and <tablename> are two [string] and are only present
+ if the Global_tables_spec flag is not set. The <column_name> is a
+ [string] and <type> is an [option] that corresponds to the description
+ (what this description is depends a bit on the context: in results to
+ selects, this will be either the user chosen alias or the selection used
+ (often a colum name, but it can be a function call too). In results to
+ a PREPARE, this will be either the name of the corresponding bind variable
+ or the column name for the variable if it is "anonymous") and type of
+ the corresponding result. The option for <type> is either a native
+ type (see below), in which case the option has no value, or a
+ 'custom' type, in which case the value is a [string] representing
+ the fully qualified class name of the type represented. Valid option
+ ids are:
+ 0x0000 Custom: the value is a [string], see above.
+ 0x0001 Ascii
+ 0x0002 Bigint
+ 0x0003 Blob
+ 0x0004 Boolean
+ 0x0005 Counter
+ 0x0006 Decimal
+ 0x0007 Double
+ 0x0008 Float
+ 0x0009 Int
+ 0x000B Timestamp
+ 0x000C Uuid
+ 0x000D Varchar
+ 0x000E Varint
+ 0x000F Timeuuid
+ 0x0010 Inet
+ 0x0011 Date
+ 0x0012 Time
+ 0x0013 Smallint
+ 0x0014 Tinyint
+ 0x0020 List: the value is an [option], representing the type
+ of the elements of the list.
+ 0x0021 Map: the value is two [option], representing the types of the
+ keys and values of the map
+ 0x0022 Set: the value is an [option], representing the type
+ of the elements of the set
+ 0x0030 UDT: the value is <ks><udt_name><n><name_1><type_1>...<name_n><type_n>
+ where:
+ - <ks> is a [string] representing the keyspace name this
+ UDT is part of.
+ - <udt_name> is a [string] representing the UDT name.
+ - <n> is a [short] representing the number of fields of
+ the UDT, and thus the number of <name_i><type_i> pairs
+ following
+ - <name_i> is a [string] representing the name of the
+ i_th field of the UDT.
+ - <type_i> is an [option] representing the type of the
+ i_th field of the UDT.
+ 0x0031 Tuple: the value is <n><type_1>...<type_n> where <n> is a [short]
+ representing the number of values in the type, and <type_i>
+ are [option] representing the type of the i_th component
+ of the tuple
+
+ - <rows_count> is an [int] representing the number of rows present in this
+ result. Those rows are serialized in the <rows_content> part.
+ - <rows_content> is composed of <row_1>...<row_m> where m is <rows_count>.
+ Each <row_i> is composed of <value_1>...<value_n> where n is
+ <columns_count> and where <value_j> is a [bytes] representing the value
+ returned for the jth column of the ith row. In other words, <rows_content>
+ is composed of (<rows_count> * <columns_count>) [bytes].
+
+
+4.2.5.3. Set_keyspace
+
+ The result to a `use` query. The body (after the kind [int]) is a single
+ [string] indicating the name of the keyspace that has been set.
+
+
+4.2.5.4. Prepared
+
+ The result to a PREPARE message. The body of a Prepared result is:
+ <id><metadata><result_metadata>
+ where:
+ - <id> is [short bytes] representing the prepared query ID.
+ - <metadata> is composed of:
+ <flags><columns_count><pk_count>[<pk_index_1>...<pk_index_n>][<global_table_spec>?<col_spec_1>...<col_spec_n>]
+ where:
+ - <flags> is an [int]. The bits of <flags> provides information on the
+ formatting of the remaining information. A flag is set if the bit
+ corresponding to its `mask` is set. Supported masks and their flags
+ are:
+ 0x0001 Global_tables_spec: if set, only one table spec (keyspace
+ and table name) is provided as <global_table_spec>. If not
+ set, <global_table_spec> is not present.
+ - <columns_count> is an [int] representing the number of bind markers
+ in the prepared statement. It defines the number of <col_spec_i>
+ elements.
+ - <pk_count> is an [int] representing the number of <pk_index_i>
+ elements to follow. If this value is zero, at least one of the
+ partition key columns in the table that the statement acts on
+ did not have a corresponding bind marker (or the bind marker
+ was wrapped in a function call).
+ - <pk_index_i> is a short that represents the index of the bind marker
+ that corresponds to the partition key column in position i.
+ For example, a <pk_index> sequence of [2, 0, 1] indicates that the
+ table has three partition key columns; the full partition key
+ can be constructed by creating a composite of the values for
+ the bind markers at index 2, at index 0, and at index 1.
+ This allows implementations with token-aware routing to correctly
+ construct the partition key without needing to inspect table
+ metadata.
+ - <global_table_spec> is present if the Global_tables_spec is set in
+ <flags>. If present, it is composed of two [string]s. The first
+ [string] is the name of the keyspace that the statement acts on.
+ The second [string] is the name of the table that the columns
+ represented by the bind markers belong to.
+ - <col_spec_i> specifies the bind markers in the prepared statement.
+ There are <column_count> such column specifications, each with the
+ following format:
+ (<ksname><tablename>)?<name><type>
+ The initial <ksname> and <tablename> are two [string] that are only
+ present if the Global_tables_spec flag is not set. The <name> field
+ is a [string] that holds the name of the bind marker (if named),
+ or the name of the column, field, or expression that the bind marker
+ corresponds to (if the bind marker is "anonymous"). The <type>
+ field is an [option] that represents the expected type of values for
+ the bind marker. See the Rows documentation (section 4.2.5.2) for
+ full details on the <type> field.
+
+ - <result_metadata> is defined exactly the same as <metadata> in the Rows
+ documentation (section 4.2.5.2). This describes the metadata for the
+ result set that will be returned when this prepared statement is executed.
+ Note that <result_metadata> may be empty (have the No_metadata flag and
+ 0 columns, See section 4.2.5.2) and will be for any query that is not a
+ Select. In fact, there is never a guarantee that this will be non-empty, so
+ implementations should protect themselves accordingly. This result metadata
+ is an optimization that allows implementations to later execute the
+ prepared statement without requesting the metadata (see the Skip_metadata
+ flag in EXECUTE). Clients can safely discard this metadata if they do not
+ want to take advantage of that optimization.
+
+ Note that the prepared query ID returned is global to the node on which the query
+ has been prepared. It can be used on any connection to that node
+ until the node is restarted (after which the query must be reprepared).
+
+4.2.5.5. Schema_change
+
+ The result to a schema altering query (creation/update/drop of a
+ keyspace/table/index). The body (after the kind [int]) is the same
+ as the body for a "SCHEMA_CHANGE" event, so 3 strings:
+ <change_type><target><options>
+ Please refer to section 4.2.6 below for the meaning of those fields.
+
+ Note that a query to create or drop an index is considered to be a change
+ to the table the index is on.
+
+
+4.2.6. EVENT
+
+ An event pushed by the server. A client will only receive events for the
+ types it has REGISTERed to. The body of an EVENT message will start with a
+ [string] representing the event type. The rest of the message depends on the
+ event type. The valid event types are:
+ - "TOPOLOGY_CHANGE": events related to change in the cluster topology.
+ Currently, events are sent when new nodes are added to the cluster, and
+ when nodes are removed. The body of the message (after the event type)
+ consists of a [string] and an [inet], corresponding respectively to the
+ type of change ("NEW_NODE" or "REMOVED_NODE") followed by the address of
+ the new/removed node.
+ - "STATUS_CHANGE": events related to change of node status. Currently,
+ up/down events are sent. The body of the message (after the event type)
+ consists of a [string] and an [inet], corresponding respectively to the
+ type of status change ("UP" or "DOWN") followed by the address of the
+ concerned node.
+ - "SCHEMA_CHANGE": events related to schema change. After the event type,
+ the rest of the message will be <change_type><target><options> where:
+ - <change_type> is a [string] representing the type of changed involved.
+ It will be one of "CREATED", "UPDATED" or "DROPPED".
+ - <target> is a [string] that can be one of "KEYSPACE", "TABLE", "TYPE",
+ "FUNCTION" or "AGGREGATE" and describes what has been modified
+ ("TYPE" stands for modifications related to user types, "FUNCTION"
+ for modifications related to user defined functions, "AGGREGATE"
+ for modifications related to user defined aggregates).
+ - <options> depends on the preceding <target>:
+ - If <target> is "KEYSPACE", then <options> will be a single [string]
+ representing the keyspace changed.
+ - If <target> is "TABLE" or "TYPE", then
+ <options> will be 2 [string]: the first one will be the keyspace
+ containing the affected object, and the second one will be the name
+ of said affected object (either the table, user type, function, or
+ aggregate name).
+ - If <target> is "FUNCTION" or "AGGREGATE", multiple arguments follow:
+ - [string] keyspace containing the user defined function / aggregate
+ - [string] the function/aggregate name
+ - [string list] one string for each argument type (as CQL type)
+
+ All EVENT messages have a streamId of -1 (Section 2.3).
+
+ Please note that "NEW_NODE" and "UP" events are sent based on internal Gossip
+ communication and as such may be sent a short delay before the binary
+ protocol server on the newly up node is fully started. Clients are thus
+ advised to wait a short time before trying to connect to the node (1 second
+ should be enough), otherwise they may experience a connection refusal at
+ first.
+
+4.2.7. AUTH_CHALLENGE
+
+ A server authentication challenge (see AUTH_RESPONSE (Section 4.1.2) for more
+ details).
+
+ The body of this message is a single [bytes] token. The details of what this
+ token contains (and when it can be null/empty, if ever) depends on the actual
+ authenticator used.
+
+ Clients are expected to answer the server challenge with an AUTH_RESPONSE
+ message.
+
+4.2.8. AUTH_SUCCESS
+
+ Indicates the success of the authentication phase. See Section 4.2.3 for more
+ details.
+
+ The body of this message is a single [bytes] token holding final information
+ from the server that the client may require to finish the authentication
+ process. What that token contains and whether it can be null depends on the
+ actual authenticator used.
+
+
+5. Compression
+
+ Frame compression is supported by the protocol, but then only the frame body
+ is compressed (the frame header should never be compressed).
+
+ Before being used, client and server must agree on a compression algorithm to
+ use, which is done in the STARTUP message. As a consequence, a STARTUP message
+ must never be compressed. However, once the STARTUP frame has been received
+ by the server, messages can be compressed (including the response to the STARTUP
+ request). Frames do not have to be compressed, however, even if compression has
+ been agreed upon (a server may only compress frames above a certain size at its
+ discretion). A frame body should be compressed if and only if the compressed
+ flag (see Section 2.2) is set.
+
+ As of version 2 of the protocol, the following compressions are available:
+ - lz4 (https://code.google.com/p/lz4/). In that, note that the first four bytes
+ of the body will be the uncompressed length (followed by the compressed
+ bytes).
+ - snappy (https://code.google.com/p/snappy/). This compression might not be
+ available as it depends on a native lib (server-side) that might not be
+ avaivable on some installations.
+
+
+6. Data Type Serialization Formats
+
+ This sections describes the serialization formats for all CQL data types
+ supported by Cassandra through the native protocol. These serialization
+ formats should be used by client drivers to encode values for EXECUTE
+ messages. Cassandra will use these formats when returning values in
+ RESULT messages.
+
+ All values are represented as [bytes] in EXECUTE and RESULT messages.
+ The [bytes] format includes an int prefix denoting the length of the value.
+ For that reason, the serialization formats described here will not include
+ a length component.
+
+ For legacy compatibility reasons, note that most non-string types support
+ "empty" values (i.e. a value with zero length). An empty value is distinct
+ from NULL, which is encoded with a negative length.
+
+ As with the rest of the native protocol, all encodings are big-endian.
+
+6.1. ascii
+
+ A sequence of bytes in the ASCII range [0, 127]. Bytes with values outside of
+ this range will result in a validation error.
+
+6.2 bigint
+
+ An eight-byte two's complement integer.
+
+6.3 blob
+
+ Any sequence of bytes.
+
+6.4 boolean
+
+ A single byte. A value of 0 denotes "false"; any other value denotes "true".
+ (However, it is recommended that a value of 1 be used to represent "true".)
+
+6.5 date
+
+ An unsigned integer representing days with epoch centered at 2^31.
+ (unix epoch January 1st, 1970).
+ A few examples:
+ 0: -5877641-06-23
+ 2^31: 1970-1-1
+ 2^32: 5881580-07-11
+
+6.6 decimal
+
+ The decimal format represents an arbitrary-precision number. It contains an
+ [int] "scale" component followed by a varint encoding (see section 6.17)
+ of the unscaled value. The encoded value represents "<unscaled>E<-scale>".
+ In other words, "<unscaled> * 10 ^ (-1 * <scale>)".
+
+6.7 double
+
+ An 8 byte floating point number in the IEEE 754 binary64 format.
+
+6.8 float
+
+ A 4 byte floating point number in the IEEE 754 binary32 format.
+
+6.9 inet
+
+ A 4 byte or 16 byte sequence denoting an IPv4 or IPv6 address, respectively.
+
+6.10 int
+
+ A 4 byte two's complement integer.
+
+6.11 list
+
+ A [int] n indicating the number of elements in the list, followed by n
+ elements. Each element is [bytes] representing the serialized value.
+
+6.12 map
+
+ A [int] n indicating the number of key/value pairs in the map, followed by
+ n entries. Each entry is composed of two [bytes] representing the key
+ and value.
+
+6.13 set
+
+ A [int] n indicating the number of elements in the set, followed by n
+ elements. Each element is [bytes] representing the serialized value.
+
+6.14 smallint
+
+ A 2 byte two's complement integer.
+
+6.15 text
+
+ A sequence of bytes conforming to the UTF-8 specifications.
+
+6.16 time
+
+ An 8 byte two's complement long representing nanoseconds since midnight.
+ Valid values are in the range 0 to 86399999999999
+
+6.17 timestamp
+
+ An 8 byte two's complement integer representing a millisecond-precision
+ offset from the unix epoch (00:00:00, January 1st, 1970). Negative values
+ represent a negative offset from the epoch.
+
+6.18 timeuuid
+
+ A 16 byte sequence representing a version 1 UUID as defined by RFC 4122.
+
+6.19 tinyint
+
+ A 1 byte two's complement integer.
+
+6.20 tuple
+
+ A sequence of [bytes] values representing the items in a tuple. The encoding
+ of each element depends on the data type for that position in the tuple.
+ Null values may be represented by using length -1 for the [bytes]
+ representation of an element.
+
+6.21 uuid
+
+ A 16 byte sequence representing any valid UUID as defined by RFC 4122.
+
+6.22 varchar
+
+ An alias of the "text" type.
+
+6.23 varint
+
+ A variable-length two's complement encoding of a signed integer.
+
+ The following examples may help implementors of this spec:
+
+ Value | Encoding
+ ------|---------
+ 0 | 0x00
+ 1 | 0x01
+ 127 | 0x7F
+ 128 | 0x0080
+ 129 | 0x0081
+ -1 | 0xFF
+ -128 | 0x80
+ -129 | 0xFF7F
+
+ Note that positive numbers must use a most-significant byte with a value
+ less than 0x80, because a most-significant bit of 1 indicates a negative
+ value. Implementors should pad positive values that have a MSB >= 0x80
+ with a leading 0x00 byte.
+
+
+7. User Defined Types
+
+ This section describes the serialization format for User defined types (UDT),
+ as described in section 4.2.5.2.
+
+ A UDT value is composed of successive [bytes] values, one for each field of the UDT
+ value (in the order defined by the type). A UDT value will generally have one value
+ for each field of the type it represents, but it is allowed to have less values than
+ the type has fields.
+
+
+8. Result paging
+
+ The protocol allows for paging the result of queries. For that, the QUERY and
+ EXECUTE messages have a <result_page_size> value that indicate the desired
+ page size in CQL3 rows.
+
+ If a positive value is provided for <result_page_size>, the result set of the
+ RESULT message returned for the query will contain at most the
+ <result_page_size> first rows of the query result. If that first page of results
+ contains the full result set for the query, the RESULT message (of kind `Rows`)
+ will have the Has_more_pages flag *not* set. However, if some results are not
+ part of the first response, the Has_more_pages flag will be set and the result
+ will contain a <paging_state> value. In that case, the <paging_state> value
+ should be used in a QUERY or EXECUTE message (that has the *same* query as
+ the original one or the behavior is undefined) to retrieve the next page of
+ results.
+
+ Only CQL3 queries that return a result set (RESULT message with a Rows `kind`)
+ support paging. For other type of queries, the <result_page_size> value is
+ ignored.
+
+ Note to client implementors:
+ - While <result_page_size> can be as low as 1, it will likely be detrimental
+ to performance to pick a value too low. A value below 100 is probably too
+ low for most use cases.
+ - Clients should not rely on the actual size of the result set returned to
+ decide if there are more results to fetch or not. Instead, they should always
+ check the Has_more_pages flag (unless they did not enable paging for the query
+ obviously). Clients should also not assert that no result will have more than
+ <result_page_size> results. While the current implementation always respects
+ the exact value of <result_page_size>, we reserve the right to return
+ slightly smaller or bigger pages in the future for performance reasons.
+ - The <paging_state> is specific to a protocol version and drivers should not
+ send a <paging_state> returned by a node using the protocol v3 to query a node
+ using the protocol v4 for instance.
+
+
+9. Error codes
+
+ Let us recall that an ERROR message is composed of <code><message>[...]
+ (see 4.2.1 for details). The supported error codes, as well as any additional
+ information the message may contain after the <message> are described below:
+ 0x0000 Server error: something unexpected happened. This indicates a
+ server-side bug.
+ 0x000A Protocol error: some client message triggered a protocol
+ violation (for instance a QUERY message is sent before a STARTUP
+ one has been sent)
+ 0x0100 Authentication error: authentication was required and failed. The
+ possible reason for failing depends on the authenticator in use,
+ which may or may not include more detail in the accompanying
+ error message.
+ 0x1000 Unavailable exception. The rest of the ERROR message body will be
+ <cl><required><alive>
+ where:
+ <cl> is the [consistency] level of the query that triggered
+ the exception.
+ <required> is an [int] representing the number of nodes that
+ should be alive to respect <cl>
+ <alive> is an [int] representing the number of replicas that
+ were known to be alive when the request had been
+ processed (since an unavailable exception has been
+ triggered, there will be <alive> < <required>)
+ 0x1001 Overloaded: the request cannot be processed because the
+ coordinator node is overloaded
+ 0x1002 Is_bootstrapping: the request was a read request but the
+ coordinator node is bootstrapping
+ 0x1003 Truncate_error: error during a truncation error.
+ 0x1100 Write_timeout: Timeout exception during a write request. The rest
+ of the ERROR message body will be
+ <cl><received><blockfor><writeType>
+ where:
+ <cl> is the [consistency] level of the query having triggered
+ the exception.
+ <received> is an [int] representing the number of nodes having
+ acknowledged the request.
+ <blockfor> is an [int] representing the number of replicas whose
+ acknowledgement is required to achieve <cl>.
+ <writeType> is a [string] that describe the type of the write
+ that timed out. The value of that string can be one
+ of:
+ - "SIMPLE": the write was a non-batched
+ non-counter write.
+ - "BATCH": the write was a (logged) batch write.
+ If this type is received, it means the batch log
+ has been successfully written (otherwise a
+ "BATCH_LOG" type would have been sent instead).
+ - "UNLOGGED_BATCH": the write was an unlogged
+ batch. No batch log write has been attempted.
+ - "COUNTER": the write was a counter write
+ (batched or not).
+ - "BATCH_LOG": the timeout occurred during the
+ write to the batch log when a (logged) batch
+ write was requested.
+ 0x1200 Read_timeout: Timeout exception during a read request. The rest
+ of the ERROR message body will be
+ <cl><received><blockfor><data_present>
+ where:
+ <cl> is the [consistency] level of the query having triggered
+ the exception.
+ <received> is an [int] representing the number of nodes having
+ answered the request.
+ <blockfor> is an [int] representing the number of replicas whose
+ response is required to achieve <cl>. Please note that
+ it is possible to have <received> >= <blockfor> if
+ <data_present> is false. Also in the (unlikely)
+ case where <cl> is achieved but the coordinator node
+ times out while waiting for read-repair acknowledgement.
+ <data_present> is a single byte. If its value is 0, it means
+ the replica that was asked for data has not
+ responded. Otherwise, the value is != 0.
+ 0x1300 Read_failure: A non-timeout exception during a read request. The rest
+ of the ERROR message body will be
+ <cl><received><blockfor><numfailures><data_present>
+ where:
+ <cl> is the [consistency] level of the query having triggered
+ the exception.
+ <received> is an [int] representing the number of nodes having
+ answered the request.
+ <blockfor> is an [int] representing the number of replicas whose
+ acknowledgement is required to achieve <cl>.
+ <numfailures> is an [int] representing the number of nodes that
+ experience a failure while executing the request.
+ <data_present> is a single byte. If its value is 0, it means
+ the replica that was asked for data had not
+ responded. Otherwise, the value is != 0.
+ 0x1400 Function_failure: A (user defined) function failed during execution.
+ The rest of the ERROR message body will be
+ <keyspace><function><arg_types>
+ where:
+ <keyspace> is the keyspace [string] of the failed function
+ <function> is the name [string] of the failed function
+ <arg_types> [string list] one string for each argument type (as CQL type) of the failed function
+ 0x1500 Write_failure: A non-timeout exception during a write request. The rest
+ of the ERROR message body will be
+ <cl><received><blockfor><numfailures><write_type>
+ where:
+ <cl> is the [consistency] level of the query having triggered
+ the exception.
+ <received> is an [int] representing the number of nodes having
+ answered the request.
+ <blockfor> is an [int] representing the number of replicas whose
+ acknowledgement is required to achieve <cl>.
+ <numfailures> is an [int] representing the number of nodes that
+ experience a failure while executing the request.
+ <writeType> is a [string] that describes the type of the write
+ that failed. The value of that string can be one
+ of:
+ - "SIMPLE": the write was a non-batched
+ non-counter write.
+ - "BATCH": the write was a (logged) batch write.
+ If this type is received, it means the batch log
+ has been successfully written (otherwise a
+ "BATCH_LOG" type would have been sent instead).
+ - "UNLOGGED_BATCH": the write was an unlogged
+ batch. No batch log write has been attempted.
+ - "COUNTER": the write was a counter write
+ (batched or not).
+ - "BATCH_LOG": the failure occured during the
+ write to the batch log when a (logged) batch
+ write was requested.
+
+ 0x2000 Syntax_error: The submitted query has a syntax error.
+ 0x2100 Unauthorized: The logged user doesn't have the right to perform
+ the query.
+ 0x2200 Invalid: The query is syntactically correct but invalid.
+ 0x2300 Config_error: The query is invalid because of some configuration issue
+ 0x2400 Already_exists: The query attempted to create a keyspace or a
+ table that was already existing. The rest of the ERROR message
+ body will be <ks><table> where:
+ <ks> is a [string] representing either the keyspace that
+ already exists, or the keyspace in which the table that
+ already exists is.
+ <table> is a [string] representing the name of the table that
+ already exists. If the query was attempting to create a
+ keyspace, <table> will be present but will be the empty
+ string.
+ 0x2500 Unprepared: Can be thrown while a prepared statement tries to be
+ executed if the provided prepared statement ID is not known by
+ this host. The rest of the ERROR message body will be [short
+ bytes] representing the unknown ID.
+
+10. Changes from v4
+
+ * Beta protocol flag for v5 native protocol is added (Section 2.2)
diff --git a/doc/source/_static/extra.css b/doc/source/_static/extra.css
index b55515e..fd3573f 100644
--- a/doc/source/_static/extra.css
+++ b/doc/source/_static/extra.css
@@ -41,3 +41,12 @@
td.rightcolumn {
padding-left: 30px;
}
+
+div#wipwarning {
+ font-size: 14px;
+ border: 1px solid #ecc;
+ color: #f66;
+ background: #ffe8e8;
+ padding: 10px 30px;
+ margin-bottom: 30px;
+}
diff --git a/doc/source/_templates/indexcontent.html b/doc/source/_templates/indexcontent.html
index a71a7e9..35c6589 100644
--- a/doc/source/_templates/indexcontent.html
+++ b/doc/source/_templates/indexcontent.html
@@ -1,5 +1,8 @@
{% extends "defindex.html" %}
{% block tables %}
+<div id="wipwarning">This documentation is currently a work-in-progress and contains a number of TODO sections.
+ <a href="{{ pathto("bugs") }}">Contributions</a> are welcome.</div>
+
<p><strong>{% trans %}Main documentation parts:{% endtrans %}</strong></p>
<table class="contentstable" align="center"><tr>
<td width="50%">
diff --git a/doc/source/_util/cql.py b/doc/source/_util/cql.py
new file mode 100644
index 0000000..b1c8cde
--- /dev/null
+++ b/doc/source/_util/cql.py
@@ -0,0 +1,267 @@
+# -*- coding: utf-8 -*-
+"""
+ CQL pygments lexer
+ ~~~~~~~~~~~~~~~~~~
+
+ Lexer for the Cassandra Query Language (CQL).
+
+ This is heavily inspired from the pygments SQL lexer (and the Postgres one in particular) but adapted to CQL
+ keywords and specificities.
+
+ TODO: This has been hacked quickly, but once it's more tested, we could submit it upstream.
+ In particular, we have alot of keywords whose meaning depends on the context and we could potentially improve
+ their handling. For instance, SET is a keyword, but also a type name (that's why currently we also consider
+ map and list as keywords, not types; we could disambiguate by looking if there is a '<' afterwards). Or things
+ like USERS, which can is used in some documentation example as a table name but is a keyword too (we could
+ only consider it a keyword if after LIST for instance). Similarly, type nanes are not reserved, so they and
+ are sometime used as column identifiers (also, timestamp is both a type and a keyword). I "think" we can
+ somewhat disambiguate through "states", but unclear how far it's worth going.
+
+ We could also add the predefined functions?
+"""
+
+import re
+
+from pygments.lexer import Lexer, RegexLexer, do_insertions, bygroups, words
+from pygments.token import Punctuation, Whitespace, Error, \
+ Text, Comment, Operator, Keyword, Name, String, Number, Generic, Literal
+from pygments.lexers import get_lexer_by_name, ClassNotFound
+from pygments.util import iteritems
+
+__all__ = [ 'CQLLexer' ]
+
+language_re = re.compile(r"\s+LANGUAGE\s+'?(\w+)'?", re.IGNORECASE)
+
+KEYWORDS = (
+ 'SELECT',
+ 'FROM',
+ 'AS',
+ 'WHERE',
+ 'AND',
+ 'KEY',
+ 'KEYS',
+ 'ENTRIES',
+ 'FULL',
+ 'INSERT',
+ 'UPDATE',
+ 'WITH',
+ 'LIMIT',
+ 'PER',
+ 'PARTITION',
+ 'USING',
+ 'USE',
+ 'DISTINCT',
+ 'COUNT',
+ 'SET',
+ 'BEGIN',
+ 'UNLOGGED',
+ 'BATCH',
+ 'APPLY',
+ 'TRUNCATE',
+ 'DELETE',
+ 'IN',
+ 'CREATE',
+ 'KEYSPACE',
+ 'SCHEMA',
+ 'KEYSPACES',
+ 'COLUMNFAMILY',
+ 'TABLE',
+ 'MATERIALIZED',
+ 'VIEW',
+ 'INDEX',
+ 'CUSTOM',
+ 'ON',
+ 'TO',
+ 'DROP',
+ 'PRIMARY',
+ 'INTO',
+ 'VALUES',
+ 'TIMESTAMP',
+ 'TTL',
+ 'CAST',
+ 'ALTER',
+ 'RENAME',
+ 'ADD',
+ 'TYPE',
+ 'COMPACT',
+ 'STORAGE',
+ 'ORDER',
+ 'BY',
+ 'ASC',
+ 'DESC',
+ 'ALLOW',
+ 'FILTERING',
+ 'IF',
+ 'IS',
+ 'CONTAINS',
+ 'GRANT',
+ 'ALL',
+ 'PERMISSION',
+ 'PERMISSIONS',
+ 'OF',
+ 'REVOKE',
+ 'MODIFY',
+ 'AUTHORIZE',
+ 'DESCRIBE',
+ 'EXECUTE',
+ 'NORECURSIVE',
+ 'MBEAN',
+ 'MBEANS',
+ 'USER',
+ 'USERS',
+ 'ROLE',
+ 'ROLES',
+ 'SUPERUSER',
+ 'NOSUPERUSER',
+ 'PASSWORD',
+ 'LOGIN',
+ 'NOLOGIN',
+ 'OPTIONS',
+ 'CLUSTERING',
+ 'TOKEN',
+ 'WRITETIME',
+ 'NULL',
+ 'NOT',
+ 'EXISTS',
+ 'MAP',
+ 'LIST',
+ 'NAN',
+ 'INFINITY',
+ 'TUPLE',
+ 'TRIGGER',
+ 'STATIC',
+ 'FROZEN',
+ 'FUNCTION',
+ 'FUNCTIONS',
+ 'AGGREGATE',
+ 'SFUNC',
+ 'STYPE',
+ 'FINALFUNC',
+ 'INITCOND',
+ 'RETURNS',
+ 'CALLED',
+ 'INPUT',
+ 'LANGUAGE',
+ 'OR',
+ 'REPLACE',
+ 'JSON',
+ 'LIKE',
+)
+
+DATATYPES = (
+ 'ASCII',
+ 'BIGINT',
+ 'BLOB',
+ 'BOOLEAN',
+ 'COUNTER',
+ 'DATE',
+ 'DECIMAL',
+ 'DOUBLE',
+ 'EMPTY',
+ 'FLOAT',
+ 'INET',
+ 'INT',
+ 'SMALLINT',
+ 'TEXT',
+ 'TIME',
+ 'TIMESTAMP',
+ 'TIMEUUID',
+ 'TINYINT',
+ 'UUID',
+ 'VARCHAR',
+ 'VARINT',
+)
+
+def language_callback(lexer, match):
+ """Parse the content of a $-string using a lexer
+
+ The lexer is chosen looking for a nearby LANGUAGE or assumed as
+ java if no LANGUAGE has been found.
+ """
+ l = None
+ m = language_re.match(lexer.text[max(0, match.start()-100):match.start()])
+ if m is not None:
+ l = lexer._get_lexer(m.group(1))
+ else:
+ l = lexer._get_lexer('java')
+
+ # 1 = $, 2 = delimiter, 3 = $
+ yield (match.start(1), String, match.group(1))
+ yield (match.start(2), String.Delimiter, match.group(2))
+ yield (match.start(3), String, match.group(3))
+ # 4 = string contents
+ if l:
+ for x in l.get_tokens_unprocessed(match.group(4)):
+ yield x
+ else:
+ yield (match.start(4), String, match.group(4))
+ # 5 = $, 6 = delimiter, 7 = $
+ yield (match.start(5), String, match.group(5))
+ yield (match.start(6), String.Delimiter, match.group(6))
+ yield (match.start(7), String, match.group(7))
+
+
+class CQLLexer(RegexLexer):
+ """
+ Lexer for the Cassandra Query Language.
+ """
+
+ name = 'Cassandra Query Language'
+ aliases = ['cql']
+ filenames = ['*.cql']
+ mimetypes = ['text/x-cql']
+
+ flags = re.IGNORECASE
+ tokens = {
+ 'root': [
+ (r'\s+', Text),
+ (r'--.*\n?', Comment.Single),
+ (r'//.*\n?', Comment.Single),
+ (r'/\*', Comment.Multiline, 'multiline-comments'),
+ (r'(' + '|'.join(s.replace(" ", "\s+")
+ for s in DATATYPES)
+ + r')\b', Name.Builtin),
+ (words(KEYWORDS, suffix=r'\b'), Keyword),
+ (r'[+*/<>=~!@#%^&|`?-]+', Operator),
+ (r'\$\d+', Name.Variable),
+
+ # Using Number instead of the more accurate Literal because the latter don't seem to e highlighted in most
+ # styles
+ (r'[0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}', Number), # UUIDs
+ (r'0x[0-9a-fA-F]+', Number), # Blobs
+
+ (r'([0-9]*\.[0-9]*|[0-9]+)(e[+-]?[0-9]+)?', Number.Float),
+ (r'[0-9]+', Number.Integer),
+ (r"((?:E|U&)?)(')", bygroups(String.Affix, String.Single), 'string'),
+ # quoted identifier
+ (r'((?:U&)?)(")', bygroups(String.Affix, String.Name), 'quoted-ident'),
+ (r'(?s)(\$)([^$]*)(\$)(.*?)(\$)(\2)(\$)', language_callback),
+ (r'[a-z_]\w*', Name),
+ (r'[;:()\[\]{},.]', Punctuation),
+ ],
+ 'multiline-comments': [
+ (r'/\*', Comment.Multiline, 'multiline-comments'),
+ (r'\*/', Comment.Multiline, '#pop'),
+ (r'[^/*]+', Comment.Multiline),
+ (r'[/*]', Comment.Multiline)
+ ],
+ 'string': [
+ (r"[^']+", String.Single),
+ (r"''", String.Single),
+ (r"'", String.Single, '#pop'),
+ ],
+ 'quoted-ident': [
+ (r'[^"]+', String.Name),
+ (r'""', String.Name),
+ (r'"', String.Name, '#pop'),
+ ],
+ }
+
+ def get_tokens_unprocessed(self, text, *args):
+ # Have a copy of the entire text to be used by `language_callback`.
+ self.text = text
+ for x in RegexLexer.get_tokens_unprocessed(self, text, *args):
+ yield x
+
+ def _get_lexer(self, lang):
+ return get_lexer_by_name(lang, **self.options)
diff --git a/doc/source/architecture/dynamo.rst b/doc/source/architecture/dynamo.rst
index d146471..a7dbb87 100644
--- a/doc/source/architecture/dynamo.rst
+++ b/doc/source/architecture/dynamo.rst
@@ -17,6 +17,8 @@
Dynamo
------
+.. _gossip:
+
Gossip
^^^^^^
diff --git a/doc/source/conf.py b/doc/source/conf.py
index 9caf188..63697aa 100644
--- a/doc/source/conf.py
+++ b/doc/source/conf.py
@@ -21,7 +21,7 @@
#
# This file is execfile()d with the current directory set to its containing
# dir.
-import re
+import re, sys, os
# Finds out the version (so we don't have to manually edit that file every
# time we change the version)
@@ -32,6 +32,10 @@
raise RuntimeException("Problem finding version in build.xml file, this shouldn't happen.")
cassandra_version = m.group(1)
+def setup(sphinx):
+ sys.path.insert(0, os.path.abspath('./source/_util'))
+ from cql import CQLLexer
+ sphinx.add_lexer("cql", CQLLexer())
# -- General configuration ------------------------------------------------
@@ -46,8 +50,13 @@
'sphinx.ext.todo',
'sphinx.ext.mathjax',
'sphinx.ext.ifconfig',
+ 'sphinx.ext.extlinks',
]
+extlinks = {
+ 'jira': ( 'https://issues.apache.org/jira/browse/CASSANDRA-%s', 'CASSANDRA-')
+}
+
# Add any paths that contain templates here, relative to this directory.
templates_path = ['_templates']
diff --git a/doc/source/cql/appendices.rst b/doc/source/cql/appendices.rst
index c4bb839..8c63a32 100644
--- a/doc/source/cql/appendices.rst
+++ b/doc/source/cql/appendices.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
Appendices
----------
diff --git a/doc/source/cql/changes.rst b/doc/source/cql/changes.rst
index 263df13..d0c51cc 100644
--- a/doc/source/cql/changes.rst
+++ b/doc/source/cql/changes.rst
@@ -14,13 +14,18 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
Changes
-------
The following describes the changes in each version of CQL.
+3.4.3
+^^^^^
+
+- Adds a ``DEFAULT UNSET`` option for ``INSERT JSON`` to ignore omitted columns (:jira:`11424`).
+
3.4.2
^^^^^
diff --git a/doc/source/cql/ddl.rst b/doc/source/cql/ddl.rst
index 7f3431a..029c1cb 100644
--- a/doc/source/cql/ddl.rst
+++ b/doc/source/cql/ddl.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _data-definition:
diff --git a/doc/source/cql/definitions.rst b/doc/source/cql/definitions.rst
index 6c3b522..e54bcd7 100644
--- a/doc/source/cql/definitions.rst
+++ b/doc/source/cql/definitions.rst
@@ -16,6 +16,8 @@
.. _UUID: https://en.wikipedia.org/wiki/Universally_unique_identifier
+.. highlight:: cql
+
Definitions
-----------
@@ -145,7 +147,7 @@
::
- — This is a comment
+ -- This is a comment
// This is a comment too
/* This is
a multi-line comment */
diff --git a/doc/source/cql/dml.rst b/doc/source/cql/dml.rst
index 989c0ca..f1c126b 100644
--- a/doc/source/cql/dml.rst
+++ b/doc/source/cql/dml.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _data-manipulation:
@@ -202,7 +202,7 @@
SELECT * FROM posts
WHERE userid = 'john doe'
- AND (blog_title, posted_at) IN (('John''s Blog', '2012-01-01), ('Extreme Chess', '2014-06-01'))
+ AND (blog_title, posted_at) IN (('John''s Blog', '2012-01-01'), ('Extreme Chess', '2014-06-01'))
The ``CONTAINS`` operator may only be used on collection columns (lists, sets, and maps). In the case of maps,
``CONTAINS`` applies to the map values. The ``CONTAINS KEY`` operator may only be used on map columns and applies to the
@@ -295,7 +295,7 @@
: [ IF NOT EXISTS ]
: [ USING `update_parameter` ( AND `update_parameter` )* ]
names_values: `names` VALUES `tuple_literal`
- json_clause: JSON `string`
+ json_clause: JSON `string` [ DEFAULT ( NULL | UNSET ) ]
names: '(' `column_name` ( ',' `column_name` )* ')'
For instance::
diff --git a/doc/source/cql/functions.rst b/doc/source/cql/functions.rst
index efcdf32..47026cd 100644
--- a/doc/source/cql/functions.rst
+++ b/doc/source/cql/functions.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _cql-functions:
@@ -39,6 +39,11 @@
evil, but no sandbox is perfect so using user-defined functions is opt-in). See the ``enable_user_defined_functions``
in ``cassandra.yaml`` to enable them.
+A function is identifier by its name:
+
+.. productionlist::
+ function_name: [ `keyspace_name` '.' ] `name`
+
.. _scalar-functions:
Scalar functions
@@ -236,15 +241,15 @@
The implicitly available ``udfContext`` field (or binding for script UDFs) provides the necessary functionality to
create new UDT and tuple values::
- CREATE TYPE custom\_type (txt text, i int);
+ CREATE TYPE custom_type (txt text, i int);
CREATE FUNCTION fct\_using\_udt ( somearg int )
RETURNS NULL ON NULL INPUT
- RETURNS custom\_type
+ RETURNS custom_type
LANGUAGE java
AS $$
UDTValue udt = udfContext.newReturnUDTValue();
- udt.setString(“txt”, “some string”);
- udt.setInt(“i”, 42);
+ udt.setString("txt", "some string");
+ udt.setInt("i", 42);
return udt;
$$;
@@ -447,25 +452,25 @@
CALLED ON NULL INPUT
RETURNS tuple
LANGUAGE java
- AS '
+ AS $$
if (val != null) {
state.setInt(0, state.getInt(0)+1);
state.setLong(1, state.getLong(1)+val.intValue());
}
return state;
- ';
+ $$;
CREATE OR REPLACE FUNCTION averageFinal (state tuple<int,bigint>)
CALLED ON NULL INPUT
RETURNS double
LANGUAGE java
- AS '
+ AS $$
double r = 0;
if (state.getInt(0) == 0) return null;
r = state.getLong(1);
r /= state.getInt(0);
- return Double.valueOf®;
- ';
+ return Double.valueOf(r);
+ $$;
CREATE OR REPLACE AGGREGATE average(int)
SFUNC averageState
diff --git a/doc/source/cql/indexes.rst b/doc/source/cql/indexes.rst
index fbe5827..81fe429 100644
--- a/doc/source/cql/indexes.rst
+++ b/doc/source/cql/indexes.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _secondary-indexes:
diff --git a/doc/source/cql/json.rst b/doc/source/cql/json.rst
index 6482fd6..539180a 100644
--- a/doc/source/cql/json.rst
+++ b/doc/source/cql/json.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _cql-json:
@@ -49,8 +49,11 @@
INSERT INTO mytable JSON '{ "\"myKey\"": 0, "value": 0}'
-Any columns which are omitted from the ``JSON`` map will be defaulted to a ``NULL`` value (which will result in a
-tombstone being created).
+By default (or if ``DEFAULT NULL`` is explicitly used), a column omitted from the ``JSON`` map will be set to ``NULL``,
+meaning that any pre-existing value for that column will be removed (resulting in a tombstone being created).
+Alternatively, if the ``DEFAULT UNSET`` directive is used after the value, omitted column values will be left unset,
+meaning that pre-existing values for those column will be preserved.
+
JSON Encoding of Cassandra Data Types
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
diff --git a/doc/source/cql/mvs.rst b/doc/source/cql/mvs.rst
index 84c18e0..aabea10 100644
--- a/doc/source/cql/mvs.rst
+++ b/doc/source/cql/mvs.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _materialized-views:
@@ -46,7 +46,7 @@
SELECT * FROM monkeySpecies
WHERE population IS NOT NULL AND species IS NOT NULL
PRIMARY KEY (population, species)
- WITH comment=‘Allow query by population instead of species’;
+ WITH comment='Allow query by population instead of species';
The ``CREATE MATERIALIZED VIEW`` statement creates a new materialized view. Each such view is a set of *rows* which
corresponds to rows which are present in the underlying, or base, table specified in the ``SELECT`` statement. A
diff --git a/doc/source/cql/security.rst b/doc/source/cql/security.rst
index aa65383..9efe27f 100644
--- a/doc/source/cql/security.rst
+++ b/doc/source/cql/security.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _cql-security:
@@ -26,6 +26,11 @@
Database Roles
^^^^^^^^^^^^^^
+CQL uses database roles to represent users and group of users. Syntactically, a role is defined by:
+
+.. productionlist::
+ role_name: `identifier` | `string`
+
.. _create-role-statement:
CREATE ROLE
diff --git a/doc/source/cql/triggers.rst b/doc/source/cql/triggers.rst
index 3bba72d..db3f53e 100644
--- a/doc/source/cql/triggers.rst
+++ b/doc/source/cql/triggers.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _cql-triggers:
diff --git a/doc/source/cql/types.rst b/doc/source/cql/types.rst
index 80cf864..e452f35 100644
--- a/doc/source/cql/types.rst
+++ b/doc/source/cql/types.rst
@@ -14,7 +14,7 @@
.. See the License for the specific language governing permissions and
.. limitations under the License.
-.. highlight:: sql
+.. highlight:: cql
.. _UUID: https://en.wikipedia.org/wiki/Universally_unique_identifier
diff --git a/doc/source/faq/index.rst b/doc/source/faq/index.rst
index 4ac0be4..d985e37 100644
--- a/doc/source/faq/index.rst
+++ b/doc/source/faq/index.rst
@@ -17,4 +17,282 @@
Frequently Asked Questions
==========================
-.. TODO: todo
+- :ref:`why-cant-list-all`
+- :ref:`what-ports`
+- :ref:`what-happens-on-joins`
+- :ref:`asynch-deletes`
+- :ref:`one-entry-ring`
+- :ref:`can-large-blob`
+- :ref:`nodetool-connection-refused`
+- :ref:`to-batch-or-not-to-batch`
+- :ref:`selinux`
+- :ref:`how-to-unsubscribe`
+- :ref:`cassandra-eats-all-my-memory`
+- :ref:`what-are-seeds`
+- :ref:`are-seeds-SPOF`
+- :ref:`why-message-dropped`
+- :ref:`oom-map-failed`
+- :ref:`what-on-same-timestamp-update`
+- :ref:`why-bootstrapping-stream-error`
+
+.. _why-cant-list-all:
+
+Why can't I set ``listen_address`` to listen on 0.0.0.0 (all my addresses)?
+---------------------------------------------------------------------------
+
+Cassandra is a gossip-based distributed system and ``listen_address`` is the address a node tells other nodes to reach
+it at. Telling other nodes "contact me on any of my addresses" is a bad idea; if different nodes in the cluster pick
+different addresses for you, Bad Things happen.
+
+If you don't want to manually specify an IP to ``listen_address`` for each node in your cluster (understandable!), leave
+it blank and Cassandra will use ``InetAddress.getLocalHost()`` to pick an address. Then it's up to you or your ops team
+to make things resolve correctly (``/etc/hosts/``, dns, etc).
+
+One exception to this process is JMX, which by default binds to 0.0.0.0 (Java bug 6425769).
+
+See :jira:`256` and :jira:`43` for more gory details.
+
+.. _what-ports:
+
+What ports does Cassandra use?
+------------------------------
+
+By default, Cassandra uses 7000 for cluster communication (7001 if SSL is enabled), 9042 for native protocol clients,
+and 7199 for JMX (and 9160 for the deprecated Thrift interface). The internode communication and native protocol ports
+are configurable in the :ref:`cassandra-yaml`. The JMX port is configurable in ``cassandra-env.sh`` (through JVM
+options). All ports are TCP.
+
+.. _what-happens-on-joins:
+
+What happens to existing data in my cluster when I add new nodes?
+-----------------------------------------------------------------
+
+When a new nodes joins a cluster, it will automatically contact the other nodes in the cluster and copy the right data
+to itself. See :ref:`topology-changes`.
+
+.. _asynch-deletes:
+
+I delete data from Cassandra, but disk usage stays the same. What gives?
+------------------------------------------------------------------------
+
+Data you write to Cassandra gets persisted to SSTables. Since SSTables are immutable, the data can't actually be removed
+when you perform a delete, instead, a marker (also called a "tombstone") is written to indicate the value's new status.
+Never fear though, on the first compaction that occurs between the data and the tombstone, the data will be expunged
+completely and the corresponding disk space recovered. See :ref:`compaction` for more detail.
+
+.. _one-entry-ring:
+
+Why does nodetool ring only show one entry, even though my nodes logged that they see each other joining the ring?
+------------------------------------------------------------------------------------------------------------------
+
+This happens when you have the same token assigned to each node. Don't do that.
+
+Most often this bites people who deploy by installing Cassandra on a VM (especially when using the Debian package, which
+auto-starts Cassandra after installation, thus generating and saving a token), then cloning that VM to other nodes.
+
+The easiest fix is to wipe the data and commitlog directories, thus making sure that each node will generate a random
+token on the next restart.
+
+.. _change-replication-factor:
+
+Can I change the replication factor (a a keyspace) on a live cluster?
+---------------------------------------------------------------------
+
+Yes, but it will require running repair (or cleanup) to change the replica count of existing data:
+
+- :ref:`Alter <alter-keyspace-statement>` the replication factor for desired keyspace (using cqlsh for instance).
+- If you're reducing the replication factor, run ``nodetool cleanup`` on the cluster to remove surplus replicated data.
+ Cleanup runs on a per-node basis.
+- If you're increasing the replication factor, run ``nodetool repair`` to ensure data is replicated according to the new
+ configuration. Repair runs on a per-replica set basis. This is an intensive process that may result in adverse cluster
+ performance. It's highly recommended to do rolling repairs, as an attempt to repair the entire cluster at once will
+ most likely swamp it.
+
+.. _can-large-blob:
+
+Can I Store (large) BLOBs in Cassandra?
+---------------------------------------
+
+Cassandra isn't optimized for large file or BLOB storage and a single ``blob`` value is always read and send to the
+client entirely. As such, storing small blobs (less than single digit MB) should not be a problem, but it is advised to
+manually split large blobs into smaller chunks.
+
+Please note in particular that by default, any value greater than 16MB will be rejected by Cassandra due the
+``max_mutation_size_in_kb`` configuration of the :ref:`cassandra-yaml` file (which default to half of
+``commitlog_segment_size_in_mb``, which itself default to 32MB).
+
+.. _nodetool-connection-refused:
+
+Nodetool says "Connection refused to host: 127.0.1.1" for any remote host. What gives?
+--------------------------------------------------------------------------------------
+
+Nodetool relies on JMX, which in turn relies on RMI, which in turn sets up its own listeners and connectors as needed on
+each end of the exchange. Normally all of this happens behind the scenes transparently, but incorrect name resolution
+for either the host connecting, or the one being connected to, can result in crossed wires and confusing exceptions.
+
+If you are not using DNS, then make sure that your ``/etc/hosts`` files are accurate on both ends. If that fails, try
+setting the ``-Djava.rmi.server.hostname=<public name>`` JVM option near the bottom of ``cassandra-env.sh`` to an
+interface that you can reach from the remote machine.
+
+.. _to-batch-or-not-to-batch:
+
+Will batching my operations speed up my bulk load?
+--------------------------------------------------
+
+No. Using batches to load data will generally just add "spikes" of latency. Use asynchronous INSERTs instead, or use
+true :ref:`bulk-loading`.
+
+An exception is batching updates to a single partition, which can be a Good Thing (as long as the size of a single batch
+stay reasonable). But never ever blindly batch everything!
+
+.. _selinux:
+
+On RHEL nodes are unable to join the ring
+-----------------------------------------
+
+Check if `SELinux <https://en.wikipedia.org/wiki/Security-Enhanced_Linux>`__ is on; if it is, turn it off.
+
+.. _how-to-unsubscribe:
+
+How do I unsubscribe from the email list?
+-----------------------------------------
+
+Send an email to ``user-unsubscribe@cassandra.apache.org``.
+
+.. _cassandra-eats-all-my-memory:
+
+Why does top report that Cassandra is using a lot more memory than the Java heap max?
+-------------------------------------------------------------------------------------
+
+Cassandra uses `Memory Mapped Files <https://en.wikipedia.org/wiki/Memory-mapped_file>`__ (mmap) internally. That is, we
+use the operating system's virtual memory system to map a number of on-disk files into the Cassandra process' address
+space. This will "use" virtual memory; i.e. address space, and will be reported by tools like top accordingly, but on 64
+bit systems virtual address space is effectively unlimited so you should not worry about that.
+
+What matters from the perspective of "memory use" in the sense as it is normally meant, is the amount of data allocated
+on brk() or mmap'd /dev/zero, which represent real memory used. The key issue is that for a mmap'd file, there is never
+a need to retain the data resident in physical memory. Thus, whatever you do keep resident in physical memory is
+essentially just there as a cache, in the same way as normal I/O will cause the kernel page cache to retain data that
+you read/write.
+
+The difference between normal I/O and mmap() is that in the mmap() case the memory is actually mapped to the process,
+thus affecting the virtual size as reported by top. The main argument for using mmap() instead of standard I/O is the
+fact that reading entails just touching memory - in the case of the memory being resident, you just read it - you don't
+even take a page fault (so no overhead in entering the kernel and doing a semi-context switch). This is covered in more
+detail `here <http://www.varnish-cache.org/trac/wiki/ArchitectNotes>`__.
+
+.. _what-are-seeds:
+
+What are seeds?
+---------------
+
+Seeds are used during startup to discover the cluster.
+
+If you configure your nodes to refer some node as seed, nodes in your ring tend to send Gossip message to seeds more
+often (also see the :ref:`section on gossip <gossip>`) than to non-seeds. In other words, seeds are worked as hubs of
+Gossip network. With seeds, each node can detect status changes of other nodes quickly.
+
+Seeds are also referred by new nodes on bootstrap to learn other nodes in ring. When you add a new node to ring, you
+need to specify at least one live seed to contact. Once a node join the ring, it learns about the other nodes, so it
+doesn't need seed on subsequent boot.
+
+You can make a seed a node at any time. There is nothing special about seed nodes. If you list the node in seed list it
+is a seed
+
+Seeds do not auto bootstrap (i.e. if a node has itself in its seed list it will not automatically transfer data to itself)
+If you want a node to do that, bootstrap it first and then add it to seeds later. If you have no data (new install) you
+do not have to worry about bootstrap at all.
+
+Recommended usage of seeds:
+
+- pick two (or more) nodes per data center as seed nodes.
+- sync the seed list to all your nodes
+
+.. _are-seeds-SPOF:
+
+Does single seed mean single point of failure?
+----------------------------------------------
+
+The ring can operate or boot without a seed; however, you will not be able to add new nodes to the cluster. It is
+recommended to configure multiple seeds in production system.
+
+.. _cant-call-jmx-method:
+
+Why can't I call jmx method X on jconsole?
+------------------------------------------
+
+Some of JMX operations use array argument and as jconsole doesn't support array argument, those operations can't be
+called with jconsole (the buttons are inactive for them). You need to write a JMX client to call such operations or need
+array-capable JMX monitoring tool.
+
+.. _why-message-dropped:
+
+Why do I see "... messages dropped ..." in the logs?
+----------------------------------------------------
+
+This is a symptom of load shedding -- Cassandra defending itself against more requests than it can handle.
+
+Internode messages which are received by a node, but do not get not to be processed within their proper timeout (see
+``read_request_timeout``, ``write_request_timeout``, ... in the :ref:`cassandra-yaml`), are dropped rather than
+processed (since the as the coordinator node will no longer be waiting for a response).
+
+For writes, this means that the mutation was not applied to all replicas it was sent to. The inconsistency will be
+repaired by read repair, hints or a manual repair. The write operation may also have timeouted as a result.
+
+For reads, this means a read request may not have completed.
+
+Load shedding is part of the Cassandra architecture, if this is a persistent issue it is generally a sign of an
+overloaded node or cluster.
+
+.. _oom-map-failed:
+
+Cassandra dies with ``java.lang.OutOfMemoryError: Map failed``
+--------------------------------------------------------------
+
+If Cassandra is dying **specifically** with the "Map failed" message, it means the OS is denying java the ability to
+lock more memory. In linux, this typically means memlock is limited. Check ``/proc/<pid of cassandra>/limits`` to verify
+this and raise it (eg, via ulimit in bash). You may also need to increase ``vm.max_map_count.`` Note that the debian
+package handles this for you automatically.
+
+
+.. _what-on-same-timestamp-update:
+
+What happens if two updates are made with the same timestamp?
+-------------------------------------------------------------
+
+Updates must be commutative, since they may arrive in different orders on different replicas. As long as Cassandra has a
+deterministic way to pick the winner (in a timestamp tie), the one selected is as valid as any other, and the specifics
+should be treated as an implementation detail. That said, in the case of a timestamp tie, Cassandra follows two rules:
+first, deletes take precedence over inserts/updates. Second, if there are two updates, the one with the lexically larger
+value is selected.
+
+.. _why-bootstrapping-stream-error:
+
+Why bootstrapping a new node fails with a "Stream failed" error?
+----------------------------------------------------------------
+
+Two main possibilities:
+
+#. the GC may be creating long pauses disrupting the streaming process
+#. compactions happening in the background hold streaming long enough that the TCP connection fails
+
+In the first case, regular GC tuning advices apply. In the second case, you need to set TCP keepalive to a lower value
+(default is very high on Linux). Try to just run the following::
+
+ $ sudo /sbin/sysctl -w net.ipv4.tcp_keepalive_time=60 net.ipv4.tcp_keepalive_intvl=60 net.ipv4.tcp_keepalive_probes=5
+
+To make those settings permanent, add them to your ``/etc/sysctl.conf`` file.
+
+Note: `GCE <https://cloud.google.com/compute/>`__'s firewall will always interrupt TCP connections that are inactive for
+more than 10 min. Running the above command is highly recommended in that environment.
+
+
+
+
+
+
+
+
+
+
+
diff --git a/doc/source/operating/bulk_loading.rst b/doc/source/operating/bulk_loading.rst
new file mode 100644
index 0000000..c8224d5
--- /dev/null
+++ b/doc/source/operating/bulk_loading.rst
@@ -0,0 +1,24 @@
+.. 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.
+
+.. highlight:: none
+
+.. _bulk-loading:
+
+Bulk Loading
+------------
+
+.. todo:: TODO
diff --git a/doc/source/operating/index.rst b/doc/source/operating/index.rst
index 6fc27c8..e2cead2 100644
--- a/doc/source/operating/index.rst
+++ b/doc/source/operating/index.rst
@@ -32,6 +32,7 @@
compression
cdc
backups
+ bulk_loading
metrics
security
hardware
diff --git a/doc/source/operating/topo_changes.rst b/doc/source/operating/topo_changes.rst
index 9d6a2ba..c42708e 100644
--- a/doc/source/operating/topo_changes.rst
+++ b/doc/source/operating/topo_changes.rst
@@ -16,6 +16,8 @@
.. highlight:: none
+.. _topology-changes:
+
Adding, replacing, moving and removing nodes
--------------------------------------------
diff --git a/ide/idea-iml-file.xml b/ide/idea-iml-file.xml
index f14fe2e..3bb51b4 100644
--- a/ide/idea-iml-file.xml
+++ b/ide/idea-iml-file.xml
@@ -28,6 +28,7 @@
<sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
<sourceFolder url="file://$MODULE_DIR$/interface/thrift/gen-java" isTestSource="false" />
<sourceFolder url="file://$MODULE_DIR$/tools/stress/src" isTestSource="false" />
+ <sourceFolder url="file://$MODULE_DIR$/tools/stress/test/unit" isTestSource="true" />
<sourceFolder url="file://$MODULE_DIR$/test/unit" isTestSource="true" />
<sourceFolder url="file://$MODULE_DIR$/test/long" isTestSource="true" />
<sourceFolder url="file://$MODULE_DIR$/test/microbench" isTestSource="true" />
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java
index b16c400..381b052 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/AuthenticationException.java
@@ -34,25 +34,12 @@
import org.apache.thrift.scheme.TupleScheme;
import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
import java.util.EnumSet;
import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* invalid authentication request (invalid keyspace, user does not exist, or credentials invalid)
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfSplit.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfSplit.java
index f765b87..b654f86 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CfSplit.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CfSplit.java
@@ -34,25 +34,12 @@
import org.apache.thrift.scheme.TupleScheme;
import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
import java.util.EnumSet;
import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Represents input splits used by hadoop ColumnFamilyRecordReaders
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
index b516a38..e24a4e4 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
@@ -34,25 +34,14 @@
import org.apache.thrift.scheme.TupleScheme;
import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
import java.util.EnumSet;
import java.util.Collections;
import java.util.BitSet;
import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Basic unit of data within a ColumnFamily.
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnDef.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnDef.java
index 951c967..409e4ac 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnDef.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnDef.java
@@ -34,25 +34,13 @@
import org.apache.thrift.scheme.TupleScheme;
import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
import java.util.EnumSet;
import java.util.Collections;
import java.util.BitSet;
import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
public class ColumnDef implements org.apache.thrift.TBase<ColumnDef, ColumnDef._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDef> {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDef");
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnOrSuperColumn.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnOrSuperColumn.java
index 2de9d0e..261d93f 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnOrSuperColumn.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnOrSuperColumn.java
@@ -34,25 +34,12 @@
import org.apache.thrift.scheme.TupleScheme;
import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
import java.util.EnumSet;
import java.util.Collections;
import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Methods for fetching rows/records from Cassandra will return either a single instance of ColumnOrSuperColumn or a list
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnPath.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnPath.java
index 627b9a0..997c1bb 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnPath.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/ColumnPath.java
@@ -34,25 +34,13 @@
import org.apache.thrift.scheme.TupleScheme;
import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
import java.util.EnumSet;
import java.util.Collections;
import java.util.BitSet;
import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* The ColumnPath is the path to a single column in Cassandra. It might make sense to think of ColumnPath and
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterColumn.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterColumn.java
index 6069218..cd9c593 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterColumn.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/CounterColumn.java
@@ -34,7 +34,6 @@
import org.apache.thrift.scheme.TupleScheme;
import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
import org.apache.thrift.EncodingUtils;
import org.apache.thrift.TException;
import org.apache.thrift.async.AsyncMethodCallback;
diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py
index d0524fe..94e8fe6 100644
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@ -848,15 +848,18 @@
try:
return open(fname, 'rb')
except IOError, e:
- printdebugmsg("Can't open %r for reading: %s" % (fname, e))
- return None
+ raise IOError("Can't open %r for reading: %s" % (fname, e))
for path in paths.split(','):
path = path.strip()
if os.path.isfile(path):
yield make_source(path)
else:
- for f in glob.glob(path):
+ result = glob.glob(path)
+ if len(result) == 0:
+ raise IOError("Can't open %r for reading: no matching file found" % (path,))
+
+ for f in result:
yield (make_source(f))
def start(self):
@@ -1269,7 +1272,11 @@
self.on_fork()
reader = self.reader
- reader.start()
+ try:
+ reader.start()
+ except IOError, exc:
+ self.outmsg.send(ImportTaskError(exc.__class__.__name__, exc.message))
+
channels = self.worker_channels
max_pending_chunks = self.max_pending_chunks
sent = 0
diff --git a/pylib/cqlshlib/test/cassconnect.py b/pylib/cqlshlib/test/cassconnect.py
index 4a1311e..0c4c0f0 100644
--- a/pylib/cqlshlib/test/cassconnect.py
+++ b/pylib/cqlshlib/test/cassconnect.py
@@ -24,15 +24,13 @@
test_keyspace_init = os.path.join(rundir, 'test_keyspace_init.cql')
-def get_cassandra_connection(cql_version=cqlsh.DEFAULT_CQLVER):
- if cql_version is None:
- cql_version = cqlsh.DEFAULT_CQLVER
+def get_cassandra_connection(cql_version=None):
conn = cql((TEST_HOST,), TEST_PORT, cql_version=cql_version, load_balancing_policy=policy)
# until the cql lib does this for us
conn.cql_version = cql_version
return conn
-def get_cassandra_cursor(cql_version=cqlsh.DEFAULT_CQLVER):
+def get_cassandra_cursor(cql_version=None):
return get_cassandra_connection(cql_version=cql_version).cursor()
TEST_KEYSPACES_CREATED = []
@@ -83,7 +81,7 @@
c.execute('DROP KEYSPACE %s' % quote_name(TEST_KEYSPACES_CREATED.pop(-1)))
@contextlib.contextmanager
-def cassandra_connection(cql_version=cqlsh.DEFAULT_CQLVER):
+def cassandra_connection(cql_version=None):
"""
Make a Cassandra CQL connection with the given CQL version and get a cursor
for it, and optionally connect to a given keyspace.
diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py
index 21eb088..df4f7e8 100644
--- a/pylib/cqlshlib/test/test_cqlsh_completion.py
+++ b/pylib/cqlshlib/test/test_cqlsh_completion.py
@@ -42,7 +42,7 @@
class CqlshCompletionCase(BaseTestCase):
def setUp(self):
- self.cqlsh_runner = testrun_cqlsh(cqlver=cqlsh.DEFAULT_CQLVER, env={'COLUMNS': '100000'})
+ self.cqlsh_runner = testrun_cqlsh(cqlver=None, env={'COLUMNS': '100000'})
self.cqlsh = self.cqlsh_runner.__enter__()
def tearDown(self):
diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py
index 8dba651..681da77 100644
--- a/pylib/cqlshlib/test/test_cqlsh_output.py
+++ b/pylib/cqlshlib/test/test_cqlsh_output.py
@@ -67,13 +67,6 @@
'Actually got: %s\ncolor code: %s'
% (tags, coloredtext.colored_version(), coloredtext.colortags()))
- def assertCqlverQueriesGiveColoredOutput(self, queries_and_expected_outputs,
- cqlver=(cqlsh.DEFAULT_CQLVER,), **kwargs):
- if not isinstance(cqlver, (tuple, list)):
- cqlver = (cqlver,)
- for ver in cqlver:
- self.assertQueriesGiveColoredOutput(queries_and_expected_outputs, cqlver=ver, **kwargs)
-
def assertQueriesGiveColoredOutput(self, queries_and_expected_outputs, **kwargs):
"""
Allow queries and expected output to be specified in structured tuples,
@@ -133,7 +126,7 @@
self.assertHasColors(c.read_to_next_prompt())
def test_count_output(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
('select count(*) from has_all_types;', """
count
MMMMM
@@ -198,7 +191,7 @@
(1 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
q = 'select COUNT(*) FROM twenty_rows_composite_table limit 1000000;'
self.assertQueriesGiveColoredOutput((
@@ -214,10 +207,10 @@
(1 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_static_cf_output(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
("select a, b from twenty_rows_table where a in ('1', '13', '2');", """
a | b
RR MM
@@ -234,7 +227,7 @@
(3 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
self.assertQueriesGiveColoredOutput((
('select * from dynamic_columns;', """
@@ -257,11 +250,11 @@
(5 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_empty_cf_output(self):
# we print the header after CASSANDRA-6910
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
('select * from empty_table;', """
lonelykey | lonelycol
RRRRRRRRR MMMMMMMMM
@@ -270,7 +263,7 @@
(0 rows)
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
q = 'select * from has_all_types where num = 999;'
@@ -284,7 +277,7 @@
(0 rows)
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_columnless_key_output(self):
q = "select a from twenty_rows_table where a in ('1', '2', '-9192');"
@@ -304,10 +297,10 @@
(2 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_numeric_output(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
('''select intcol, bigintcol, varintcol \
from has_all_types \
where num in (0, 1, 2, 3, 4);''', """
@@ -353,7 +346,7 @@
(5 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_timestamp_output(self):
self.assertQueriesGiveColoredOutput((
@@ -390,7 +383,7 @@
pass
def test_boolean_output(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
('select num, booleancol from has_all_types where num in (0, 1, 2, 3);', """
num | booleancol
RRR MMMMMMMMMM
@@ -409,11 +402,11 @@
(4 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_null_output(self):
# column with metainfo but no values
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
("select k, c, notthere from undefined_values_table where k in ('k1', 'k2');", """
k | c | notthere
R M MMMMMMMM
@@ -428,7 +421,7 @@
(2 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
# all-columns, including a metainfo column has no values (cql3)
self.assertQueriesGiveColoredOutput((
@@ -446,10 +439,10 @@
(2 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_string_output_ascii(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
("select * from ascii_with_special_chars where k in (0, 1, 2, 3);", r"""
k | val
R MMM
@@ -468,7 +461,7 @@
(4 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_string_output_utf8(self):
# many of these won't line up visually here, to keep the source code
@@ -477,7 +470,7 @@
# terminals, but the color-checking machinery here will still treat
# it as one character, so those won't seem to line up visually either.
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
("select * from utf8_with_special_chars where k in (0, 1, 2, 3, 4, 5, 6);", u"""
k | val
R MMM
@@ -502,10 +495,10 @@
(7 rows)
nnnnnnnn
""".encode('utf-8')),
- ), cqlver=cqlsh.DEFAULT_CQLVER, env={'LANG': 'en_US.UTF-8'})
+ ), env={'LANG': 'en_US.UTF-8'})
def test_blob_output(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
("select num, blobcol from has_all_types where num in (0, 1, 2, 3);", r"""
num | blobcol
RRR MMMMMMM
@@ -524,10 +517,10 @@
(4 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_prompt(self):
- with testrun_cqlsh(tty=True, keyspace=None, cqlver=cqlsh.DEFAULT_CQLVER) as c:
+ with testrun_cqlsh(tty=True, keyspace=None) as c:
self.assertTrue(c.output_header.splitlines()[-1].endswith('cqlsh> '))
c.send('\n')
@@ -559,8 +552,7 @@
"RRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRR")
def test_describe_keyspace_output(self):
- fullcqlver = cqlsh.DEFAULT_CQLVER
- with testrun_cqlsh(tty=True, cqlver=fullcqlver) as c:
+ with testrun_cqlsh(tty=True) as c:
ks = get_keyspace()
qks = quote_name(ks)
for cmd in ('describe keyspace', 'desc keyspace'):
@@ -568,7 +560,7 @@
for semicolon in ('', ';'):
fullcmd = cmd + (' ' if givename else '') + givename + semicolon
desc = c.cmd_and_response(fullcmd)
- self.check_describe_keyspace_output(desc, givename or qks, fullcqlver)
+ self.check_describe_keyspace_output(desc, givename or qks)
# try to actually execute that last keyspace description, with a
# new keyspace name
@@ -577,7 +569,7 @@
statements = split_cql_commands(copy_desc)
do_drop = True
- with cassandra_cursor(cql_version=fullcqlver) as curs:
+ with cassandra_cursor() as curs:
try:
for stmt in statements:
cqlshlog.debug('TEST EXEC: %s' % stmt)
@@ -587,7 +579,7 @@
if do_drop:
curs.execute('drop keyspace %s' % quote_name(new_ks_name))
- def check_describe_keyspace_output(self, output, qksname, fullcqlver):
+ def check_describe_keyspace_output(self, output, qksname):
expected_bits = [r'(?im)^CREATE KEYSPACE %s WITH\b' % re.escape(qksname),
r';\s*$',
r'\breplication = {\'class\':']
@@ -636,7 +628,7 @@
""" % quote_name(get_keyspace()))
- with testrun_cqlsh(tty=True, cqlver=cqlsh.DEFAULT_CQLVER) as c:
+ with testrun_cqlsh(tty=True) as c:
for cmdword in ('describe table', 'desc columnfamily'):
for semicolon in (';', ''):
output = c.cmd_and_response('%s has_all_types%s' % (cmdword, semicolon))
@@ -654,7 +646,7 @@
ks = get_keyspace()
- with testrun_cqlsh(tty=True, keyspace=None, cqlver=cqlsh.DEFAULT_CQLVER) as c:
+ with testrun_cqlsh(tty=True, keyspace=None) as c:
# when not in a keyspace
for cmdword in ('DESCRIBE COLUMNFAMILIES', 'desc tables'):
@@ -705,7 +697,7 @@
\n
'''
- with testrun_cqlsh(tty=True, keyspace=None, cqlver=cqlsh.DEFAULT_CQLVER) as c:
+ with testrun_cqlsh(tty=True, keyspace=None) as c:
# not in a keyspace
for semicolon in ('', ';'):
@@ -793,7 +785,7 @@
pass
def test_user_types_output(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
("select addresses from users;", r"""
addresses
MMMMMMMMM
@@ -808,8 +800,8 @@
(2 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
- self.assertCqlverQueriesGiveColoredOutput((
+ ))
+ self.assertQueriesGiveColoredOutput((
("select phone_numbers from users;", r"""
phone_numbers
MMMMMMMMMMMMM
@@ -824,10 +816,10 @@
(2 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
def test_user_types_with_collections(self):
- self.assertCqlverQueriesGiveColoredOutput((
+ self.assertQueriesGiveColoredOutput((
("select info from songs;", r"""
info
MMMM
@@ -840,8 +832,8 @@
(1 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
- self.assertCqlverQueriesGiveColoredOutput((
+ ))
+ self.assertQueriesGiveColoredOutput((
("select tags from songs;", r"""
tags
MMMM
@@ -854,4 +846,4 @@
(1 rows)
nnnnnnnn
"""),
- ), cqlver=cqlsh.DEFAULT_CQLVER)
+ ))
diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g
index 16b2ac4..a65b4f5 100644
--- a/src/antlr/Lexer.g
+++ b/src/antlr/Lexer.g
@@ -195,6 +195,8 @@
K_REPLACE: R E P L A C E;
K_JSON: J S O N;
+K_DEFAULT: D E F A U L T;
+K_UNSET: U N S E T;
K_LIKE: L I K E;
// Case-insensitive alpha characters
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index f61f464..e762bde 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -359,12 +359,14 @@
@init {
Attributes.Raw attrs = new Attributes.Raw();
boolean ifNotExists = false;
+ boolean defaultUnset = false;
}
: val=jsonValue
+ ( K_DEFAULT ( K_NULL | ( { defaultUnset = true; } K_UNSET) ) )?
( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
( usingClause[attrs] )?
{
- $expr = new UpdateStatement.ParsedInsertJson(cf, attrs, val, ifNotExists);
+ $expr = new UpdateStatement.ParsedInsertJson(cf, attrs, val, defaultUnset, ifNotExists);
}
;
@@ -775,22 +777,24 @@
TableAttributes attrs = new TableAttributes();
Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames = new HashMap<ColumnDefinition.Raw, ColumnDefinition.Raw>();
List<AlterTableStatementColumn> colNameList = new ArrayList<AlterTableStatementColumn>();
+ Long deleteTimestamp = null;
}
: K_ALTER K_COLUMNFAMILY cf=columnFamilyName
( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; } { colNameList.add(new AlterTableStatementColumn(id,v)); }
| K_ADD ( (id=cident v=comparatorType b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id,v,b1)); })
| ('(' id1=cident v1=comparatorType b1=cfisStatic { colNameList.add(new AlterTableStatementColumn(id1,v1,b1)); }
( ',' idn=cident vn=comparatorType bn=cfisStatic { colNameList.add(new AlterTableStatementColumn(idn,vn,bn)); } )* ')' ) ) { type = AlterTableStatement.Type.ADD; }
- | K_DROP ( id=cident { colNameList.add(new AlterTableStatementColumn(id)); }
- | ('(' id1=cident { colNameList.add(new AlterTableStatementColumn(id1)); }
- ( ',' idn=cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') ) { type = AlterTableStatement.Type.DROP; }
+ | K_DROP ( ( id=cident { colNameList.add(new AlterTableStatementColumn(id)); }
+ | ('(' id1=cident { colNameList.add(new AlterTableStatementColumn(id1)); }
+ ( ',' idn=cident { colNameList.add(new AlterTableStatementColumn(idn)); } )* ')') )
+ ( K_USING K_TIMESTAMP t=INTEGER { deleteTimestamp = Long.parseLong(Constants.Literal.integer($t.text).getText()); })? ) { type = AlterTableStatement.Type.DROP; }
| K_WITH properties[attrs] { type = AlterTableStatement.Type.OPTS; }
| K_RENAME { type = AlterTableStatement.Type.RENAME; }
id1=cident K_TO toId1=cident { renames.put(id1, toId1); }
( K_AND idn=cident K_TO toIdn=cident { renames.put(idn, toIdn); } )*
)
{
- $expr = new AlterTableStatement(cf, type, colNameList, attrs, renames);
+ $expr = new AlterTableStatement(cf, type, colNameList, attrs, renames, deleteTimestamp);
}
;
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 619ecf8..a6c11d2 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -41,16 +41,10 @@
import org.apache.cassandra.serializers.SetSerializer;
import org.apache.cassandra.serializers.UTF8Serializer;
import org.apache.cassandra.service.ClientState;
-import java.lang.management.ManagementFactory;
-
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
import org.apache.cassandra.cql3.QueryOptions;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.UntypedResultSet.Row;
import org.apache.cassandra.service.QueryState;
import org.apache.cassandra.transport.messages.ResultMessage;
import org.apache.cassandra.utils.ByteBufferUtil;
diff --git a/src/java/org/apache/cassandra/auth/CassandraLoginModule.java b/src/java/org/apache/cassandra/auth/CassandraLoginModule.java
index 2ccf962..d208300 100644
--- a/src/java/org/apache/cassandra/auth/CassandraLoginModule.java
+++ b/src/java/org/apache/cassandra/auth/CassandraLoginModule.java
@@ -145,7 +145,7 @@
AuthenticatedUser user = authenticator.legacyAuthenticate(credentials);
// Only actual users should be allowed to authenticate for JMX
if (user.isAnonymous() || user.isSystem())
- throw new AuthenticationException("Invalid user");
+ throw new AuthenticationException(String.format("Invalid user %s", user.getName()));
// The LOGIN privilege is required to authenticate - c.f. ClientState::login
if (!DatabaseDescriptor.getRoleManager().canLogin(user.getPrimaryRole()))
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index 3714523..74eb10d 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -87,7 +87,7 @@
{
String hash = cache.get(username);
if (!BCrypt.checkpw(password, hash))
- throw new AuthenticationException("Username and/or password are incorrect");
+ throw new AuthenticationException(String.format("Provided username %s and/or password are incorrect", username));
return new AuthenticatedUser(username);
}
@@ -95,13 +95,13 @@
{
// the credentials were somehow invalid - either a non-existent role, or one without a defined password
if (e.getCause() instanceof NoSuchCredentialsException)
- throw new AuthenticationException("Username and/or password are incorrect");
+ throw new AuthenticationException(String.format("Provided username %s and/or password are incorrect", username));
// an unanticipated exception occured whilst querying the credentials table
if (e.getCause() instanceof RequestExecutionException)
{
logger.trace("Error performing internal authentication", e);
- throw new AuthenticationException(e.getMessage());
+ throw new AuthenticationException(String.format("Error during authentication of user %s : %s", username, e.getMessage()));
}
throw new RuntimeException(e);
@@ -180,7 +180,7 @@
String password = credentials.get(PASSWORD_KEY);
if (password == null)
- throw new AuthenticationException(String.format("Required key '%s' is missing", PASSWORD_KEY));
+ throw new AuthenticationException(String.format("Required key '%s' is missing for provided username %s", PASSWORD_KEY, username));
return authenticate(username, password);
}
diff --git a/src/java/org/apache/cassandra/auth/RoleResource.java b/src/java/org/apache/cassandra/auth/RoleResource.java
index e994233..bf3b0ec 100644
--- a/src/java/org/apache/cassandra/auth/RoleResource.java
+++ b/src/java/org/apache/cassandra/auth/RoleResource.java
@@ -20,7 +20,6 @@
import java.util.Set;
import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import org.apache.commons.lang3.StringUtils;
diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
index 3bac1f6..f19b19f 100644
--- a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
+++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
@@ -29,7 +29,6 @@
import javax.management.MBeanServer;
import javax.management.MalformedObjectNameException;
import javax.management.ObjectName;
-import javax.management.remote.MBeanServerForwarder;
import javax.security.auth.Subject;
import com.google.common.annotations.VisibleForTesting;
@@ -40,9 +39,7 @@
import org.apache.cassandra.auth.*;
import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
/**
* Provides a proxy interface to the platform's MBeanServer instance to perform
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index cb2ad8a..b98ad53 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -44,7 +44,7 @@
import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.io.util.ChecksummedRandomAccessReader.CorruptFileException;
+import org.apache.cassandra.io.util.CorruptFileException;
import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -91,7 +91,7 @@
public InputStream getInputStream(File dataPath, File crcPath) throws IOException
{
- return new ChecksummedRandomAccessReader.Builder(dataPath, crcPath).build();
+ return ChecksummedRandomAccessReader.open(dataPath, crcPath);
}
public OutputStream getOutputStream(File dataPath, File crcPath)
diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java
index e6296bd..d91d9fd 100644
--- a/src/java/org/apache/cassandra/cache/ChunkCache.java
+++ b/src/java/org/apache/cassandra/cache/ChunkCache.java
@@ -31,7 +31,6 @@
import com.github.benmanes.caffeine.cache.*;
import com.codahale.metrics.Timer;
import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.io.compress.BufferType;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.io.util.*;
import org.apache.cassandra.metrics.CacheMissMetrics;
@@ -184,7 +183,7 @@
return instance.wrap(file);
}
- public void invalidatePosition(SegmentedFile dfile, long position)
+ public void invalidatePosition(FileHandle dfile, long position)
{
if (!(dfile.rebuffererFactory() instanceof CachingRebufferer))
return;
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
index f47d8ac..530f46e 100644
--- a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@ -29,8 +29,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.concurrent.SimpleCondition;
import org.apache.cassandra.utils.JVMStabilityInspector;
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index 1fb0690..92cbbf4 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -22,9 +22,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-
import static org.apache.cassandra.tracing.Tracing.isTracing;
/**
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
index 367dc7c..6577b3d 100644
--- a/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
@@ -19,7 +19,6 @@
package org.apache.cassandra.concurrent;
import org.apache.cassandra.service.ClientWarn;
-import org.apache.cassandra.tracing.TraceState;
import org.apache.cassandra.tracing.Tracing;
public interface ExecutorLocal<T>
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 64abf00..84b8da6 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -59,7 +59,7 @@
stages.put(Stage.TRACING, tracingExecutor());
}
- private static ExecuteOnlyExecutor tracingExecutor()
+ private static LocalAwareExecutorService tracingExecutor()
{
RejectedExecutionHandler reh = new RejectedExecutionHandler()
{
@@ -68,13 +68,13 @@
MessagingService.instance().incrementDroppedMessages(MessagingService.Verb._TRACE);
}
};
- return new ExecuteOnlyExecutor(1,
- 1,
- KEEPALIVE,
- TimeUnit.SECONDS,
- new ArrayBlockingQueue<Runnable>(1000),
- new NamedThreadFactory(Stage.TRACING.getJmxName()),
- reh);
+ return new TracingExecutor(1,
+ 1,
+ KEEPALIVE,
+ TimeUnit.SECONDS,
+ new ArrayBlockingQueue<Runnable>(1000),
+ new NamedThreadFactory(Stage.TRACING.getJmxName()),
+ reh);
}
private static JMXEnabledThreadPoolExecutor multiThreadedStage(Stage stage, int numThreads)
@@ -112,17 +112,12 @@
}
}
- public final static Runnable NO_OP_TASK = () -> {};
-
/**
- * A TPE that disallows submit so that we don't need to worry about unwrapping exceptions on the
- * tracing stage. See CASSANDRA-1123 for background. We allow submitting NO_OP tasks, to allow
- * a final wait on pending trace events since typically the tracing executor is single-threaded, see
- * CASSANDRA-11465.
+ * The executor used for tracing.
*/
- private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
+ private static class TracingExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
{
- public ExecuteOnlyExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
+ public TracingExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
{
super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
}
@@ -137,28 +132,5 @@
{
execute(command);
}
-
- @Override
- public Future<?> submit(Runnable task)
- {
- if (task.equals(NO_OP_TASK))
- {
- assert getMaximumPoolSize() == 1 : "Cannot wait for pending tasks if running more than 1 thread";
- return super.submit(task);
- }
- throw new UnsupportedOperationException();
- }
-
- @Override
- public <T> Future<T> submit(Runnable task, T result)
- {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public <T> Future<T> submit(Callable<T> task)
- {
- throw new UnsupportedOperationException();
- }
}
}
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 4de4f7b..0aaf064 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -32,6 +32,8 @@
import com.google.common.base.Objects;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Iterators;
import com.google.common.collect.Sets;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.commons.lang3.builder.HashCodeBuilder;
@@ -46,6 +48,7 @@
import org.apache.cassandra.cql3.statements.CreateTableStatement;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
+import org.apache.cassandra.db.filter.ColumnFilter;
import org.apache.cassandra.db.marshal.*;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.exceptions.ConfigurationException;
@@ -121,6 +124,9 @@
public final DataResource resource;
+ //For hot path serialization it's often easier to store this info here
+ private volatile ColumnFilter allColumnFilter;
+
/*
* All of these methods will go away once CFMetaData becomes completely immutable.
*/
@@ -293,9 +299,12 @@
this.clusteringColumns = clusteringColumns;
this.partitionColumns = partitionColumns;
- this.serializers = new Serializers(this);
- this.resource = DataResource.table(ksName, cfName);
+ //This needs to happen before serializers are set
+ //because they use comparator.subtypes()
rebuild();
+
+ this.resource = DataResource.table(ksName, cfName);
+ this.serializers = new Serializers(this);
}
// This rebuild informations that are intrinsically duplicate of the table definition but
@@ -304,7 +313,7 @@
{
this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
- Map<ByteBuffer, ColumnDefinition> newColumnMetadata = new HashMap<>();
+ Map<ByteBuffer, ColumnDefinition> newColumnMetadata = Maps.newHashMapWithExpectedSize(partitionKeyColumns.size() + clusteringColumns.size() + partitionColumns.size());
for (ColumnDefinition def : partitionKeyColumns)
newColumnMetadata.put(def.name.bytes, def);
for (ColumnDefinition def : clusteringColumns)
@@ -322,6 +331,8 @@
if (isCompactTable())
this.compactValueColumn = CompactTables.getCompactValueColumn(partitionColumns, isSuper());
+
+ this.allColumnFilter = ColumnFilter.all(this);
}
public Indexes getIndexes()
@@ -329,6 +340,11 @@
return indexes;
}
+ public ColumnFilter getAllColumnFilter()
+ {
+ return allColumnFilter;
+ }
+
public static CFMetaData create(String ksName,
String name,
UUID cfId,
@@ -377,9 +393,9 @@
partitioner);
}
- private static List<AbstractType<?>> extractTypes(List<ColumnDefinition> clusteringColumns)
+ public static List<AbstractType<?>> extractTypes(Iterable<ColumnDefinition> clusteringColumns)
{
- List<AbstractType<?>> types = new ArrayList<>(clusteringColumns.size());
+ List<AbstractType<?>> types = new ArrayList<>();
for (ColumnDefinition def : clusteringColumns)
types.add(def.type);
return types;
@@ -964,9 +980,12 @@
return removed;
}
- public void recordColumnDrop(ColumnDefinition def)
+ /**
+ * Adds the column definition as a dropped column, recording the drop with the provided timestamp.
+ */
+ public void recordColumnDrop(ColumnDefinition def, long timeMicros)
{
- droppedColumns.put(def.name.bytes, new DroppedColumn(def.name.toString(), def.type, FBUtilities.timestampMicros()));
+ droppedColumns.put(def.name.bytes, new DroppedColumn(def.name.toString(), def.type, timeMicros));
}
public void renameColumn(ColumnIdentifier from, ColumnIdentifier to) throws InvalidRequestException
@@ -1260,7 +1279,7 @@
public Set<String> usedColumnNames()
{
- Set<String> usedNames = new HashSet<>();
+ Set<String> usedNames = Sets.newHashSetWithExpectedSize(partitionKeys.size() + clusteringColumns.size() + staticColumns.size() + regularColumns.size());
for (Pair<ColumnIdentifier, AbstractType> p : partitionKeys)
usedNames.add(p.left.toString());
for (Pair<ColumnIdentifier, AbstractType> p : clusteringColumns)
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 05d80a5..caf3925 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -46,7 +46,10 @@
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy;
+import org.apache.cassandra.io.util.SsdDiskOptimizationStrategy;
import org.apache.cassandra.locator.*;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.scheduler.IRequestScheduler;
@@ -108,6 +111,8 @@
private static EncryptionContext encryptionContext;
private static boolean hasLoggedConfig;
+ private static DiskOptimizationStrategy diskOptimizationStrategy;
+
public static void forceStaticInitialization() {}
static
{
@@ -124,6 +129,15 @@
{
applyConfig(loadConfig());
}
+ switch (conf.disk_optimization_strategy)
+ {
+ case ssd:
+ diskOptimizationStrategy = new SsdDiskOptimizationStrategy(conf.disk_optimization_page_cross_chance);
+ break;
+ case spinning:
+ diskOptimizationStrategy = new SpinningDiskOptimizationStrategy();
+ break;
+ }
}
catch (Exception e)
{
@@ -1433,6 +1447,11 @@
return conf.commitlog_max_compression_buffers_in_pool;
}
+ public static void setCommitLogMaxCompressionBuffersPerPool(int buffers)
+ {
+ conf.commitlog_max_compression_buffers_in_pool = buffers;
+ }
+
public static int getMaxMutationSize()
{
return conf.max_mutation_size_in_kb * 1024;
@@ -1873,15 +1892,9 @@
return conf.buffer_pool_use_heap_if_exhausted;
}
- public static Config.DiskOptimizationStrategy getDiskOptimizationStrategy()
+ public static DiskOptimizationStrategy getDiskOptimizationStrategy()
{
- return conf.disk_optimization_strategy;
- }
-
- @VisibleForTesting
- public static void setDiskOptimizationStrategy(Config.DiskOptimizationStrategy strategy)
- {
- conf.disk_optimization_strategy = strategy;
+ return diskOptimizationStrategy;
}
public static double getDiskOptimizationEstimatePercentile()
@@ -1889,17 +1902,6 @@
return conf.disk_optimization_estimate_percentile;
}
- public static double getDiskOptimizationPageCrossChance()
- {
- return conf.disk_optimization_page_cross_chance;
- }
-
- @VisibleForTesting
- public static void setDiskOptimizationPageCrossChance(double chance)
- {
- conf.disk_optimization_page_cross_chance = chance;
- }
-
public static long getTotalCommitlogSpaceInMB()
{
return conf.commitlog_total_space_in_mb;
@@ -2229,6 +2231,6 @@
public static int searchConcurrencyFactor()
{
- return Integer.valueOf(System.getProperty("cassandra.search_concurrency_factor", "1"));
+ return Integer.parseInt(System.getProperty("cassandra.search_concurrency_factor", "1"));
}
}
diff --git a/src/java/org/apache/cassandra/config/ParameterizedClass.java b/src/java/org/apache/cassandra/config/ParameterizedClass.java
index 6c7996a..be1b302 100644
--- a/src/java/org/apache/cassandra/config/ParameterizedClass.java
+++ b/src/java/org/apache/cassandra/config/ParameterizedClass.java
@@ -21,7 +21,6 @@
import java.util.Map;
import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableMap;
public class ParameterizedClass
{
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index ee1f137..dd42779 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -314,6 +314,11 @@
return getCFMetaData(descriptor.ksname, descriptor.cfname);
}
+ public int getNumberOfTables()
+ {
+ return cfIdMap.size();
+ }
+
public ViewDefinition getView(String keyspaceName, String viewName)
{
assert keyspaceName != null;
@@ -606,7 +611,7 @@
public void dropKeyspace(String ksName)
{
KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
- String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
+ String snapshotName = Keyspace.getTimestampedSnapshotNameWithPrefix(ksName, ColumnFamilyStore.SNAPSHOT_DROP_PREFIX);
CompactionManager.instance.interruptCompactionFor(ksm.tablesAndViews(), true);
@@ -685,7 +690,7 @@
CompactionManager.instance.interruptCompactionFor(Collections.singleton(cfm), true);
if (DatabaseDescriptor.isAutoSnapshot())
- cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
+ cfs.snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(cfs.name, ColumnFamilyStore.SNAPSHOT_DROP_PREFIX));
Keyspace.open(ksName).dropCf(cfm.cfId);
MigrationManager.instance.notifyDropColumnFamily(cfm);
diff --git a/src/java/org/apache/cassandra/cql3/AbstractMarker.java b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
index 3689ed1..7ffedef 100644
--- a/src/java/org/apache/cassandra/cql3/AbstractMarker.java
+++ b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.cql3;
-import java.util.Collections;
import java.util.List;
import org.apache.cassandra.cql3.functions.Function;
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index a443f23..534a2a0 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -18,11 +18,8 @@
package org.apache.cassandra.cql3;
import java.nio.ByteBuffer;
-import java.util.Collections;
import java.util.List;
-import com.google.common.collect.Iterables;
-
import org.apache.cassandra.cql3.functions.Function;
import org.apache.cassandra.db.LivenessInfo;
import org.apache.cassandra.db.marshal.Int32Type;
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 467c672..a17b0e1 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -18,7 +18,6 @@
package org.apache.cassandra.cql3;
import java.nio.ByteBuffer;
-import java.util.List;
import java.util.Locale;
import java.util.concurrent.ConcurrentMap;
import java.util.regex.Matcher;
@@ -28,12 +27,7 @@
import com.google.common.collect.MapMaker;
import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.selection.Selectable;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.ObjectSizes;
import org.apache.cassandra.utils.memory.AbstractAllocator;
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index 913ea97..f108e8b 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -40,6 +40,32 @@
STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX;
}
+ private static class UnsetLiteral extends Term.Raw
+ {
+ public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
+ {
+ return UNSET_VALUE;
+ }
+
+ public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+ {
+ return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+ }
+
+ public String getText()
+ {
+ return "";
+ }
+
+ public AbstractType<?> getExactTypeIfKnown(String keyspace)
+ {
+ return null;
+ }
+ }
+
+ // We don't have "unset" literal in the syntax, but it's used implicitely for JSON "DEFAULT UNSET" option
+ public static final UnsetLiteral UNSET_LITERAL = new UnsetLiteral();
+
public static final Value UNSET_VALUE = new Value(ByteBufferUtil.UNSET_BYTE_BUFFER);
private static class NullLiteral extends Term.Raw
diff --git a/src/java/org/apache/cassandra/cql3/FieldIdentifier.java b/src/java/org/apache/cassandra/cql3/FieldIdentifier.java
index 5e0601c..9f72fc4 100644
--- a/src/java/org/apache/cassandra/cql3/FieldIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/FieldIdentifier.java
@@ -22,7 +22,6 @@
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.serializers.MarshalException;
/**
diff --git a/src/java/org/apache/cassandra/cql3/Json.java b/src/java/org/apache/cassandra/cql3/Json.java
index 298cde7..2e67a1e 100644
--- a/src/java/org/apache/cassandra/cql3/Json.java
+++ b/src/java/org/apache/cassandra/cql3/Json.java
@@ -18,7 +18,6 @@
package org.apache.cassandra.cql3;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.*;
import org.apache.cassandra.config.CFMetaData;
@@ -112,7 +111,7 @@
*/
public static abstract class Prepared
{
- public abstract Term.Raw getRawTermForColumn(ColumnDefinition def);
+ public abstract Term.Raw getRawTermForColumn(ColumnDefinition def, boolean defaultUnset);
}
/**
@@ -127,10 +126,12 @@
this.columnMap = columnMap;
}
- public Term.Raw getRawTermForColumn(ColumnDefinition def)
+ public Term.Raw getRawTermForColumn(ColumnDefinition def, boolean defaultUnset)
{
Term value = columnMap.get(def.name);
- return value == null ? Constants.NULL_LITERAL : new ColumnValue(value);
+ return value == null
+ ? (defaultUnset ? Constants.UNSET_LITERAL : Constants.NULL_LITERAL)
+ : new ColumnValue(value);
}
}
@@ -148,9 +149,9 @@
this.columns = columns;
}
- public RawDelayedColumnValue getRawTermForColumn(ColumnDefinition def)
+ public RawDelayedColumnValue getRawTermForColumn(ColumnDefinition def, boolean defaultUnset)
{
- return new RawDelayedColumnValue(this, def);
+ return new RawDelayedColumnValue(this, def, defaultUnset);
}
}
@@ -199,17 +200,19 @@
{
private final PreparedMarker marker;
private final ColumnDefinition column;
+ private final boolean defaultUnset;
- public RawDelayedColumnValue(PreparedMarker prepared, ColumnDefinition column)
+ public RawDelayedColumnValue(PreparedMarker prepared, ColumnDefinition column, boolean defaultUnset)
{
this.marker = prepared;
this.column = column;
+ this.defaultUnset = defaultUnset;
}
@Override
public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
{
- return new DelayedColumnValue(marker, column);
+ return new DelayedColumnValue(marker, column, defaultUnset);
}
@Override
@@ -236,11 +239,13 @@
{
private final PreparedMarker marker;
private final ColumnDefinition column;
+ private final boolean defaultUnset;
- public DelayedColumnValue(PreparedMarker prepared, ColumnDefinition column)
+ public DelayedColumnValue(PreparedMarker prepared, ColumnDefinition column, boolean defaultUnset)
{
this.marker = prepared;
this.column = column;
+ this.defaultUnset = defaultUnset;
}
@Override
@@ -259,7 +264,9 @@
public Terminal bind(QueryOptions options) throws InvalidRequestException
{
Term term = options.getJsonColumnValue(marker.bindIndex, column.name, marker.columns);
- return term == null ? null : term.bind(options);
+ return term == null
+ ? (defaultUnset ? Constants.UNSET_VALUE : null)
+ : term.bind(options);
}
@Override
@@ -285,10 +292,16 @@
Map<ColumnIdentifier, Term> columnMap = new HashMap<>(expectedReceivers.size());
for (ColumnSpecification spec : expectedReceivers)
{
+ // We explicitely test containsKey() because the value itself can be null, and we want to distinguish an
+ // explicit null value from no value
+ if (!valueMap.containsKey(spec.name.toString()))
+ continue;
+
Object parsedJsonObject = valueMap.remove(spec.name.toString());
if (parsedJsonObject == null)
{
- columnMap.put(spec.name, null);
+ // This is an explicit user null
+ columnMap.put(spec.name, Constants.NULL_VALUE);
}
else
{
diff --git a/src/java/org/apache/cassandra/cql3/Operation.java b/src/java/org/apache/cassandra/cql3/Operation.java
index 7d7d7b3..85a1eb2 100644
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.cql3;
-import java.nio.ByteBuffer;
import java.util.List;
import org.apache.cassandra.config.CFMetaData;
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 222204b..4e7323e 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -33,7 +33,6 @@
import org.slf4j.LoggerFactory;
import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
-import com.googlecode.concurrentlinkedhashmap.EntryWeigher;
import org.antlr.runtime.*;
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.config.DatabaseDescriptor;
@@ -140,6 +139,33 @@
}
}
+ public static void preloadPreparedStatement()
+ {
+ ClientState clientState = ClientState.forInternalCalls();
+ int count = 0;
+ for (Pair<String, String> useKeyspaceAndCQL : SystemKeyspace.loadPreparedStatements())
+ {
+ try
+ {
+ clientState.setKeyspace(useKeyspaceAndCQL.left);
+ prepare(useKeyspaceAndCQL.right, clientState, false);
+ count++;
+ }
+ catch (RequestValidationException e)
+ {
+ logger.warn("prepared statement recreation error: {}", useKeyspaceAndCQL.right, e);
+ }
+ }
+ logger.info("Preloaded {} prepared statements", count);
+ }
+
+ @VisibleForTesting
+ public static void clearPrepraredStatements()
+ {
+ preparedStatements.clear();
+ thriftPreparedStatements.clear();
+ }
+
private static QueryState internalQueryState()
{
return InternalStateInstance.INSTANCE.queryState;
@@ -446,6 +472,7 @@
queryString.substring(0, 200)));
MD5Digest statementId = computeId(queryString, keyspace);
preparedStatements.put(statementId, prepared);
+ SystemKeyspace.writePreparedStatement(keyspace, statementId, queryString);
return new ResultMessage.Prepared(statementId, prepared);
}
}
@@ -555,14 +582,50 @@
private static class MigrationSubscriber extends MigrationListener
{
- private void removeInvalidPreparedStatements(String ksName, String cfName)
+ private static void removeInvalidPreparedStatements(String ksName, String cfName)
{
removeInvalidPreparedStatements(internalStatements.values().iterator(), ksName, cfName);
- removeInvalidPreparedStatements(preparedStatements.values().iterator(), ksName, cfName);
+ removeInvalidPersistentPreparedStatements(preparedStatements.entrySet().iterator(), ksName, cfName);
removeInvalidPreparedStatements(thriftPreparedStatements.values().iterator(), ksName, cfName);
}
- private void removeInvalidPreparedStatements(Iterator<ParsedStatement.Prepared> iterator, String ksName, String cfName)
+ private static void removeInvalidPreparedStatementsForFunction(String ksName, String functionName)
+ {
+ Predicate<Function> matchesFunction = f -> ksName.equals(f.name().keyspace) && functionName.equals(f.name().name);
+
+ for (Iterator<Map.Entry<MD5Digest, ParsedStatement.Prepared>> iter = preparedStatements.entrySet().iterator();
+ iter.hasNext();)
+ {
+ Map.Entry<MD5Digest, ParsedStatement.Prepared> pstmt = iter.next();
+ if (Iterables.any(pstmt.getValue().statement.getFunctions(), matchesFunction))
+ {
+ SystemKeyspace.removePreparedStatement(pstmt.getKey());
+ iter.remove();
+ }
+ }
+
+
+ Iterators.removeIf(internalStatements.values().iterator(),
+ statement -> Iterables.any(statement.statement.getFunctions(), matchesFunction));
+
+ Iterators.removeIf(thriftPreparedStatements.values().iterator(),
+ statement -> Iterables.any(statement.statement.getFunctions(), matchesFunction));
+ }
+
+ private static void removeInvalidPersistentPreparedStatements(Iterator<Map.Entry<MD5Digest, ParsedStatement.Prepared>> iterator,
+ String ksName, String cfName)
+ {
+ while (iterator.hasNext())
+ {
+ Map.Entry<MD5Digest, ParsedStatement.Prepared> entry = iterator.next();
+ if (shouldInvalidate(ksName, cfName, entry.getValue().statement)) {
+ SystemKeyspace.removePreparedStatement(entry.getKey());
+ iterator.remove();
+ }
+ }
+ }
+
+ private static void removeInvalidPreparedStatements(Iterator<ParsedStatement.Prepared> iterator, String ksName, String cfName)
{
while (iterator.hasNext())
{
@@ -571,7 +634,7 @@
}
}
- private boolean shouldInvalidate(String ksName, String cfName, CQLStatement statement)
+ private static boolean shouldInvalidate(String ksName, String cfName, CQLStatement statement)
{
String statementKsName;
String statementCfName;
@@ -621,7 +684,7 @@
// in case there are other overloads, we have to remove all overloads since argument type
// matching may change (due to type casting)
if (Schema.instance.getKSMetaData(ksName).functions.get(new FunctionName(ksName, functionName)).size() > 1)
- removeAllInvalidPreparedStatementsForFunction(ksName, functionName);
+ removeInvalidPreparedStatementsForFunction(ksName, functionName);
}
public void onUpdateColumnFamily(String ksName, String cfName, boolean affectsStatements)
@@ -637,7 +700,7 @@
// the new definition is picked (the function is resolved at preparation time).
// TODO: if the function has multiple overload, we could invalidate only the statement refering to the overload
// that was updated. This requires a few changes however and probably doesn't matter much in practice.
- removeAllInvalidPreparedStatementsForFunction(ksName, functionName);
+ removeInvalidPreparedStatementsForFunction(ksName, functionName);
}
public void onUpdateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
@@ -646,7 +709,7 @@
// the new definition is picked (the function is resolved at preparation time).
// TODO: if the function has multiple overload, we could invalidate only the statement refering to the overload
// that was updated. This requires a few changes however and probably doesn't matter much in practice.
- removeAllInvalidPreparedStatementsForFunction(ksName, aggregateName);
+ removeInvalidPreparedStatementsForFunction(ksName, aggregateName);
}
public void onDropKeyspace(String ksName)
@@ -663,27 +726,12 @@
public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
{
- removeAllInvalidPreparedStatementsForFunction(ksName, functionName);
+ removeInvalidPreparedStatementsForFunction(ksName, functionName);
}
public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
{
- removeAllInvalidPreparedStatementsForFunction(ksName, aggregateName);
- }
-
- private static void removeAllInvalidPreparedStatementsForFunction(String ksName, String functionName)
- {
- removeInvalidPreparedStatementsForFunction(internalStatements.values().iterator(), ksName, functionName);
- removeInvalidPreparedStatementsForFunction(preparedStatements.values().iterator(), ksName, functionName);
- removeInvalidPreparedStatementsForFunction(thriftPreparedStatements.values().iterator(), ksName, functionName);
- }
-
- private static void removeInvalidPreparedStatementsForFunction(Iterator<ParsedStatement.Prepared> statements,
- final String ksName,
- final String functionName)
- {
- Predicate<Function> matchesFunction = f -> ksName.equals(f.name().keyspace) && functionName.equals(f.name().name);
- Iterators.removeIf(statements, statement -> Iterables.any(statement.statement.getFunctions(), matchesFunction));
+ removeInvalidPreparedStatementsForFunction(ksName, aggregateName);
}
}
}
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 9010b20..e9a0f42 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -590,7 +590,7 @@
size += CBUtil.sizeOfString(m.names.get(0).cfName);
}
- if (m.partitionKeyBindIndexes != null && version >= 4)
+ if (m.partitionKeyBindIndexes != null && version >= Server.VERSION_4)
size += 4 + 2 * m.partitionKeyBindIndexes.length;
for (ColumnSpecification name : m.names)
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index ba9ddb6..267832d 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -26,7 +26,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.cql3.Term.MultiColumnRaw;
import org.apache.cassandra.cql3.functions.Function;
import org.apache.cassandra.db.marshal.*;
import org.apache.cassandra.exceptions.InvalidRequestException;
diff --git a/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java b/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
index 0c4f2e2..7256c66 100644
--- a/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
@@ -19,7 +19,6 @@
import java.math.BigDecimal;
import java.math.BigInteger;
-import java.math.RoundingMode;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
index 87f5019..34c6cc9 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
@@ -191,7 +191,7 @@
// javaParamTypes is just the Java representation for argTypes resp. argDataTypes
TypeToken<?>[] javaParamTypes = UDHelper.typeTokens(argCodecs, calledOnNullInput);
- // javaReturnType is just the Java representation for returnType resp. returnDataType
+ // javaReturnType is just the Java representation for returnType resp. returnTypeCodec
TypeToken<?> javaReturnType = returnCodec.getJavaType();
// put each UDF in a separate package to prevent cross-UDF code access
@@ -222,7 +222,10 @@
s = body;
break;
case "arguments":
- s = generateArguments(javaParamTypes, argNames);
+ s = generateArguments(javaParamTypes, argNames, false);
+ break;
+ case "arguments_aggregate":
+ s = generateArguments(javaParamTypes, argNames, true);
break;
case "argument_list":
s = generateArgumentList(javaParamTypes, argNames);
@@ -326,7 +329,7 @@
}
}
- if (nonSyntheticMethodCount != 2 || cls.getDeclaredConstructors().length != 1)
+ if (nonSyntheticMethodCount != 3 || cls.getDeclaredConstructors().length != 1)
throw new InvalidRequestException("Check your source to not define additional Java methods or constructors");
MethodType methodType = MethodType.methodType(void.class)
.appendParameterTypes(TypeCodec.class, TypeCodec[].class, UDFContext.class);
@@ -364,6 +367,10 @@
return javaUDF.executeImpl(protocolVersion, params);
}
+ protected Object executeAggregateUserDefined(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ return javaUDF.executeAggregateImpl(protocolVersion, firstParam, params);
+ }
private static int countNewlines(StringBuilder javaSource)
{
@@ -417,22 +424,48 @@
return code.toString();
}
- private static String generateArguments(TypeToken<?>[] paramTypes, List<ColumnIdentifier> argNames)
+ /**
+ * Generate Java source code snippet for the arguments part to call the UDF implementation function -
+ * i.e. the {@code private #return_type# #execute_internal_name#(#argument_list#)} function
+ * (see {@code JavaSourceUDF.txt} template file for details).
+ * <p>
+ * This method generates the arguments code snippet for both {@code executeImpl} and
+ * {@code executeAggregateImpl}. General signature for both is the {@code protocolVersion} and
+ * then all UDF arguments. For aggregation UDF calls the first argument is always unserialized as
+ * that is the state variable.
+ * </p>
+ * <p>
+ * An example output for {@code executeImpl}:
+ * {@code (double) super.compose_double(protocolVersion, 0, params.get(0)), (double) super.compose_double(protocolVersion, 1, params.get(1))}
+ * </p>
+ * <p>
+ * Similar output for {@code executeAggregateImpl}:
+ * {@code firstParam, (double) super.compose_double(protocolVersion, 1, params.get(1))}
+ * </p>
+ */
+ private static String generateArguments(TypeToken<?>[] paramTypes, List<ColumnIdentifier> argNames, boolean forAggregate)
{
StringBuilder code = new StringBuilder(64 * paramTypes.length);
for (int i = 0; i < paramTypes.length; i++)
{
if (i > 0)
+ // add separator, if not the first argument
code.append(",\n");
+ // add comment only if trace is enabled
if (logger.isTraceEnabled())
code.append(" /* parameter '").append(argNames.get(i)).append("' */\n");
- code
- // cast to Java type
- .append(" (").append(javaSourceName(paramTypes[i])).append(") ")
+ // cast to Java type
+ code.append(" (").append(javaSourceName(paramTypes[i])).append(") ");
+
+ if (forAggregate && i == 0)
+ // special case for aggregations where the state variable (1st arg to state + final function and
+ // return value from state function) is not re-serialized
+ code.append("firstParam");
+ else
// generate object representation of input parameter (call UDFunction.compose)
- .append(composeMethod(paramTypes[i])).append("(protocolVersion, ").append(i).append(", params.get(").append(i).append("))");
+ code.append(composeMethod(paramTypes[i])).append("(protocolVersion, ").append(i).append(", params.get(").append(forAggregate ? i - 1 : i).append("))");
}
return code.toString();
}
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java b/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java
index 7410f1f..56a7ced 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java
@@ -45,6 +45,8 @@
protected abstract ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params);
+ protected abstract Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params);
+
protected Object compose(int protocolVersion, int argIndex, ByteBuffer value)
{
return UDFunction.compose(argCodecs, protocolVersion, argIndex, value);
diff --git a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
index b524163..8c15dc9 100644
--- a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
@@ -177,6 +177,29 @@
for (int i = 0; i < params.length; i++)
params[i] = compose(protocolVersion, i, parameters.get(i));
+ Object result = executeScriptInternal(params);
+
+ return decompose(protocolVersion, result);
+ }
+
+ /**
+ * Like {@link #executeUserDefined(int, List)} but the first parameter is already in non-serialized form.
+ * Remaining parameters (2nd paramters and all others) are in {@code parameters}.
+ * This is used to prevent superfluous (de)serialization of the state of aggregates.
+ * Means: scalar functions of aggregates are called using this variant.
+ */
+ protected Object executeAggregateUserDefined(int protocolVersion, Object firstParam, List<ByteBuffer> parameters)
+ {
+ Object[] params = new Object[argTypes.size()];
+ params[0] = firstParam;
+ for (int i = 1; i < params.length; i++)
+ params[i] = compose(protocolVersion, i, parameters.get(i - 1));
+
+ return executeScriptInternal(params);
+ }
+
+ private Object executeScriptInternal(Object[] params)
+ {
ScriptContext scriptContext = new SimpleScriptContext();
scriptContext.setAttribute("javax.script.filename", this.name.toString(), ScriptContext.ENGINE_SCOPE);
Bindings bindings = scriptContext.getBindings(ScriptContext.ENGINE_SCOPE);
@@ -251,7 +274,7 @@
}
}
- return decompose(protocolVersion, result);
+ return result;
}
private final class UDFContextWrapper extends AbstractJSObject
diff --git a/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
index 623feba..79ebfaf 100644
--- a/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
@@ -27,7 +27,6 @@
import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.serializers.TimestampSerializer;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.UUIDGen;
diff --git a/src/java/org/apache/cassandra/cql3/functions/ToJsonFct.java b/src/java/org/apache/cassandra/cql3/functions/ToJsonFct.java
index bcb4559..c7391da 100644
--- a/src/java/org/apache/cassandra/cql3/functions/ToJsonFct.java
+++ b/src/java/org/apache/cassandra/cql3/functions/ToJsonFct.java
@@ -23,8 +23,6 @@
import org.apache.cassandra.utils.ByteBufferUtil;
import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
index 52b8163..6570ba8 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
@@ -24,6 +24,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.datastax.driver.core.TypeCodec;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.schema.Functions;
@@ -36,7 +37,9 @@
{
protected static final Logger logger = LoggerFactory.getLogger(UDAggregate.class);
- protected final AbstractType<?> stateType;
+ private final AbstractType<?> stateType;
+ private final TypeCodec stateTypeCodec;
+ private final TypeCodec returnTypeCodec;
protected final ByteBuffer initcond;
private final ScalarFunction stateFunction;
private final ScalarFunction finalFunction;
@@ -52,6 +55,8 @@
this.stateFunction = stateFunc;
this.finalFunction = finalFunc;
this.stateType = stateFunc != null ? stateFunc.returnType() : null;
+ this.stateTypeCodec = stateType != null ? UDHelper.codecFor(UDHelper.driverType(stateType)) : null;
+ this.returnTypeCodec = returnType != null ? UDHelper.codecFor(UDHelper.driverType(returnType)) : null;
this.initcond = initcond;
}
@@ -68,7 +73,7 @@
List<AbstractType<?>> stateTypes = new ArrayList<>(argTypes.size() + 1);
stateTypes.add(stateType);
stateTypes.addAll(argTypes);
- List<AbstractType<?>> finalTypes = Collections.<AbstractType<?>>singletonList(stateType);
+ List<AbstractType<?>> finalTypes = Collections.singletonList(stateType);
return new UDAggregate(name,
argTypes,
returnType,
@@ -81,7 +86,7 @@
List<AbstractType<?>> argTypes,
AbstractType<?> returnType,
ByteBuffer initcond,
- final InvalidRequestException reason)
+ InvalidRequestException reason)
{
return new UDAggregate(name, argTypes, returnType, null, null, initcond)
{
@@ -150,48 +155,55 @@
private long stateFunctionCount;
private long stateFunctionDuration;
- private ByteBuffer state;
- {
- reset();
- }
+ private Object state;
+ private boolean needsInit = true;
public void addInput(int protocolVersion, List<ByteBuffer> values) throws InvalidRequestException
{
+ if (needsInit)
+ {
+ state = initcond != null ? UDHelper.deserialize(stateTypeCodec, protocolVersion, initcond.duplicate()) : null;
+ stateFunctionDuration = 0;
+ stateFunctionCount = 0;
+ needsInit = false;
+ }
+
long startTime = System.nanoTime();
stateFunctionCount++;
- List<ByteBuffer> fArgs = new ArrayList<>(values.size() + 1);
- fArgs.add(state);
- fArgs.addAll(values);
if (stateFunction instanceof UDFunction)
{
UDFunction udf = (UDFunction)stateFunction;
- if (udf.isCallableWrtNullable(fArgs))
- state = udf.execute(protocolVersion, fArgs);
+ if (udf.isCallableWrtNullable(values))
+ state = udf.executeForAggregate(protocolVersion, state, values);
}
else
{
- state = stateFunction.execute(protocolVersion, fArgs);
+ throw new UnsupportedOperationException("UDAs only support UDFs");
}
stateFunctionDuration += (System.nanoTime() - startTime) / 1000;
}
public ByteBuffer compute(int protocolVersion) throws InvalidRequestException
{
+ assert !needsInit;
+
// final function is traced in UDFunction
Tracing.trace("Executed UDA {}: {} call(s) to state function {} in {}\u03bcs", name(), stateFunctionCount, stateFunction.name(), stateFunctionDuration);
if (finalFunction == null)
- return state;
+ return UDFunction.decompose(stateTypeCodec, protocolVersion, state);
- List<ByteBuffer> fArgs = Collections.singletonList(state);
- ByteBuffer result = finalFunction.execute(protocolVersion, fArgs);
- return result;
+ if (finalFunction instanceof UDFunction)
+ {
+ UDFunction udf = (UDFunction)finalFunction;
+ Object result = udf.executeForAggregate(protocolVersion, state, Collections.emptyList());
+ return UDFunction.decompose(returnTypeCodec, protocolVersion, result);
+ }
+ throw new UnsupportedOperationException("UDAs only support UDFs");
}
public void reset()
{
- state = initcond != null ? initcond.duplicate() : null;
- stateFunctionDuration = 0;
- stateFunctionCount = 0;
+ needsInit = true;
}
};
}
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java b/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
index cfaa70f..7d28fcd 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
@@ -108,6 +108,13 @@
// the executeImpl method - ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
return new ExecuteImplVisitor(errors);
}
+ if ("executeAggregateImpl".equals(name) && "(ILjava/lang/Object;Ljava/util/List;)Ljava/lang/Object;".equals(desc))
+ {
+ if (Opcodes.ACC_PROTECTED != access)
+ errors.add("executeAggregateImpl not protected");
+ // the executeImpl method - ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
+ return new ExecuteImplVisitor(errors);
+ }
if ("<clinit>".equals(name))
{
errors.add("static initializer declared");
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 6e8d187..70d459f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -28,6 +28,7 @@
import java.util.HashSet;
import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@@ -258,12 +259,22 @@
return Executors.newSingleThreadExecutor();
}
+ protected Object executeAggregateUserDefined(int protocolVersion, Object firstParam, List<ByteBuffer> parameters)
+ {
+ throw broken();
+ }
+
public ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> parameters)
{
- throw new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully "
- + "for the following reason: %s. Please see the server log for details",
- this,
- reason.getMessage()));
+ throw broken();
+ }
+
+ private InvalidRequestException broken()
+ {
+ return new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully "
+ + "for the following reason: %s. Please see the server log for details",
+ this,
+ reason.getMessage()));
}
};
}
@@ -301,6 +312,44 @@
}
}
+ /**
+ * Like {@link #execute(int, List)} but the first parameter is already in non-serialized form.
+ * Remaining parameters (2nd paramters and all others) are in {@code parameters}.
+ * This is used to prevent superfluous (de)serialization of the state of aggregates.
+ * Means: scalar functions of aggregates are called using this variant.
+ */
+ public final Object executeForAggregate(int protocolVersion, Object firstParam, List<ByteBuffer> parameters)
+ {
+ assertUdfsEnabled(language);
+
+ if (!calledOnNullInput && firstParam == null || !isCallableWrtNullable(parameters))
+ return null;
+
+ long tStart = System.nanoTime();
+ parameters = makeEmptyParametersNull(parameters);
+
+ try
+ {
+ // Using async UDF execution is expensive (adds about 100us overhead per invocation on a Core-i7 MBPr).
+ Object result = DatabaseDescriptor.enableUserDefinedFunctionsThreads()
+ ? executeAggregateAsync(protocolVersion, firstParam, parameters)
+ : executeAggregateUserDefined(protocolVersion, firstParam, parameters);
+ Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
+ return result;
+ }
+ catch (InvalidRequestException e)
+ {
+ throw e;
+ }
+ catch (Throwable t)
+ {
+ logger.debug("Invocation of user-defined function '{}' failed", this, t);
+ if (t instanceof VirtualMachineError)
+ throw (VirtualMachineError) t;
+ throw FunctionExecutionException.create(this, t);
+ }
+ }
+
public static void assertUdfsEnabled(String language)
{
if (!DatabaseDescriptor.enableUserDefinedFunctions())
@@ -344,10 +393,31 @@
{
ThreadIdAndCpuTime threadIdAndCpuTime = new ThreadIdAndCpuTime();
- Future<ByteBuffer> future = executor().submit(() -> {
+ return async(threadIdAndCpuTime, () -> {
threadIdAndCpuTime.setup();
return executeUserDefined(protocolVersion, parameters);
});
+ }
+
+ /**
+ * Like {@link #executeAsync(int, List)} but the first parameter is already in non-serialized form.
+ * Remaining parameters (2nd paramters and all others) are in {@code parameters}.
+ * This is used to prevent superfluous (de)serialization of the state of aggregates.
+ * Means: scalar functions of aggregates are called using this variant.
+ */
+ private Object executeAggregateAsync(int protocolVersion, Object firstParam, List<ByteBuffer> parameters)
+ {
+ ThreadIdAndCpuTime threadIdAndCpuTime = new ThreadIdAndCpuTime();
+
+ return async(threadIdAndCpuTime, () -> {
+ threadIdAndCpuTime.setup();
+ return executeAggregateUserDefined(protocolVersion, firstParam, parameters);
+ });
+ }
+
+ private <T> T async(ThreadIdAndCpuTime threadIdAndCpuTime, Callable<T> callable)
+ {
+ Future<T> future = executor().submit(callable);
try
{
@@ -445,6 +515,8 @@
protected abstract ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> parameters);
+ protected abstract Object executeAggregateUserDefined(int protocolVersion, Object firstParam, List<ByteBuffer> parameters);
+
public boolean isAggregate()
{
return false;
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
index dc349d9..8e2d744 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
@@ -152,7 +152,12 @@
*/
public final boolean hasContains()
{
- return restrictions.stream().anyMatch(SingleRestriction::isContains);
+ for (SingleRestriction restriction : restrictions)
+ {
+ if (restriction.isContains())
+ return true;
+ }
+ return false;
}
/**
@@ -163,7 +168,12 @@
*/
public final boolean hasSlice()
{
- return restrictions.stream().anyMatch(SingleRestriction::isSlice);
+ for (SingleRestriction restriction : restrictions)
+ {
+ if (restriction.isSlice())
+ return true;
+ }
+ return false;
}
/**
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
index 2648f62..bb943d5 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
@@ -18,7 +18,8 @@
package org.apache.cassandra.cql3.restrictions;
import java.util.*;
-import java.util.stream.Stream;
+
+import com.google.common.collect.AbstractIterator;
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.cql3.QueryOptions;
@@ -53,14 +54,21 @@
*/
protected final TreeMap<ColumnDefinition, SingleRestriction> restrictions;
+ /**
+ * {@code true} if it contains multi-column restrictions, {@code false} otherwise.
+ */
+ private final boolean hasMultiColumnRestrictions;
+
public RestrictionSet()
{
- this(new TreeMap<ColumnDefinition, SingleRestriction>(COLUMN_DEFINITION_COMPARATOR));
+ this(new TreeMap<ColumnDefinition, SingleRestriction>(COLUMN_DEFINITION_COMPARATOR), false);
}
- private RestrictionSet(TreeMap<ColumnDefinition, SingleRestriction> restrictions)
+ private RestrictionSet(TreeMap<ColumnDefinition, SingleRestriction> restrictions,
+ boolean hasMultiColumnRestrictions)
{
this.restrictions = restrictions;
+ this.hasMultiColumnRestrictions = hasMultiColumnRestrictions;
}
@Override
@@ -76,24 +84,11 @@
return new ArrayList<>(restrictions.keySet());
}
- public Stream<SingleRestriction> stream()
- {
- return new LinkedHashSet<>(restrictions.values()).stream();
- }
-
@Override
public void addFunctionsTo(List<Function> functions)
{
- Restriction previous = null;
- for (Restriction restriction : restrictions.values())
- {
- // For muti-column restriction, we can have multiple time the same restriction.
- if (!restriction.equals(previous))
- {
- previous = restriction;
- restriction.addFunctionsTo(functions);
- }
- }
+ for (Restriction restriction : this)
+ restriction.addFunctionsTo(functions);
}
@Override
@@ -118,7 +113,7 @@
{
// RestrictionSet is immutable so we need to clone the restrictions map.
TreeMap<ColumnDefinition, SingleRestriction> newRestrictions = new TreeMap<>(this.restrictions);
- return new RestrictionSet(mergeRestrictions(newRestrictions, restriction));
+ return new RestrictionSet(mergeRestrictions(newRestrictions, restriction), hasMultiColumnRestrictions || restriction.isMultiColumn());
}
private TreeMap<ColumnDefinition, SingleRestriction> mergeRestrictions(TreeMap<ColumnDefinition, SingleRestriction> restrictions,
@@ -246,7 +241,8 @@
@Override
public Iterator<SingleRestriction> iterator()
{
- return new LinkedHashSet<>(restrictions.values()).iterator();
+ Iterator<SingleRestriction> iterator = restrictions.values().iterator();
+ return hasMultiColumnRestrictions ? new DistinctIterator<>(iterator) : iterator;
}
/**
@@ -255,7 +251,12 @@
*/
public final boolean hasIN()
{
- return stream().anyMatch(SingleRestriction::isIN);
+ for (SingleRestriction restriction : this)
+ {
+ if (restriction.isIN())
+ return true;
+ }
+ return false;
}
/**
@@ -266,6 +267,49 @@
*/
public final boolean hasOnlyEqualityRestrictions()
{
- return stream().allMatch(p -> p.isEQ() || p.isIN());
+ for (SingleRestriction restriction : this)
+ {
+ if (!restriction.isEQ() && !restriction.isIN())
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * {@code Iterator} decorator that removes duplicates in an ordered one.
+ *
+ * @param iterator the decorated iterator
+ * @param <E> the iterator element type.
+ */
+ private static final class DistinctIterator<E> extends AbstractIterator<E>
+ {
+ /**
+ * The decorated iterator.
+ */
+ private final Iterator<E> iterator;
+
+ /**
+ * The previous element.
+ */
+ private E previous;
+
+ public DistinctIterator(Iterator<E> iterator)
+ {
+ this.iterator = iterator;
+ }
+
+ protected E computeNext()
+ {
+ while(iterator.hasNext())
+ {
+ E next = iterator.next();
+ if (!next.equals(previous))
+ {
+ previous = next;
+ return next;
+ }
+ }
+ return endOfData();
+ }
}
}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index f8a39ee..62b48b3 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@ -478,7 +478,7 @@
{
usesSecondaryIndexing = true;
}
- else
+ else if (!allowFiltering)
{
List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
@@ -488,7 +488,7 @@
ColumnDefinition clusteringColumn = clusteringColumns.get(i);
ColumnDefinition restrictedColumn = restrictedColumns.get(i);
- if (!clusteringColumn.equals(restrictedColumn) && !allowFiltering)
+ if (!clusteringColumn.equals(restrictedColumn))
{
throw invalidRequest("PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
restrictedColumn.name,
diff --git a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
index 55ff50f..687b4c7 100644
--- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@ -23,7 +23,6 @@
import org.apache.cassandra.cql3.QueryOptions;
import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.db.marshal.UserType;
import org.apache.cassandra.exceptions.InvalidRequestException;
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 1f1f07b..fa78042 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -21,7 +21,6 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
-import java.nio.ByteBuffer;
import org.apache.commons.lang3.text.StrBuilder;
import org.apache.cassandra.config.CFMetaData;
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 2a11d27..f547376 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -21,7 +21,6 @@
import java.util.*;
import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
index 4cfdefb..5072066 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
@@ -23,8 +23,6 @@
import java.util.*;
import java.util.stream.Collectors;
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.collect.*;
diff --git a/src/java/org/apache/cassandra/cql3/selection/TermSelector.java b/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
index 5aa4522..539826a 100644
--- a/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
@@ -19,10 +19,7 @@
import java.nio.ByteBuffer;
-import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.AssignmentTestable;
-import org.apache.cassandra.cql3.ColumnIdentifier;
import org.apache.cassandra.cql3.ColumnSpecification;
import org.apache.cassandra.cql3.QueryOptions;
import org.apache.cassandra.cql3.Term;
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index afe2776..1d1cbc3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -24,9 +24,7 @@
import org.apache.cassandra.auth.Permission;
import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.*;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.marshal.AbstractType;
@@ -41,6 +39,7 @@
import org.apache.cassandra.service.ClientState;
import org.apache.cassandra.service.MigrationManager;
import org.apache.cassandra.transport.Event;
+import org.apache.cassandra.utils.*;
import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
@@ -55,18 +54,21 @@
private final TableAttributes attrs;
private final Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames;
private final List<AlterTableStatementColumn> colNameList;
+ private final Long deleteTimestamp;
public AlterTableStatement(CFName name,
Type type,
List<AlterTableStatementColumn> colDataList,
TableAttributes attrs,
- Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames)
+ Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames,
+ Long deleteTimestamp)
{
super(name);
this.oType = type;
this.colNameList = colDataList;
this.attrs = attrs;
this.renames = renames;
+ this.deleteTimestamp = deleteTimestamp == null ? FBUtilities.timestampMicros() : deleteTimestamp;
}
public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -253,15 +255,15 @@
for (ColumnDefinition columnDef : cfm.partitionColumns())
{
if (columnDef.name.equals(columnName))
- {
- toDelete = columnDef;
- break;
- }
+ {
+ toDelete = columnDef;
+ break;
+ }
}
- assert toDelete != null;
- cfm.removeColumnDefinition(toDelete);
- cfm.recordColumnDrop(toDelete);
- break;
+ assert toDelete != null;
+ cfm.removeColumnDefinition(toDelete);
+ cfm.recordColumnDrop(toDelete, deleteTimestamp);
+ break;
}
// If the dropped column is required by any secondary indexes
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
index 813effe..480709f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
@@ -19,7 +19,6 @@
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.ColumnIdentifier;
public class AlterTableStatementColumn
{
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 2739c2e..14638e2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -22,6 +22,7 @@
import java.util.concurrent.TimeUnit;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.helpers.MessageFormatter;
@@ -554,7 +555,7 @@
public Map<UUID, PartitionColumns> build()
{
- Map<UUID, PartitionColumns> m = new HashMap<>();
+ Map<UUID, PartitionColumns> m = Maps.newHashMapWithExpectedSize(perTableBuilders.size());
for (Map.Entry<UUID, PartitionColumns.Builder> p : perTableBuilders.entrySet())
m.put(p.getKey(), p.getValue().build());
return m;
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index f899247..2526f79 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -22,6 +22,7 @@
import com.google.common.base.Optional;
import com.google.common.base.Strings;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -173,7 +174,7 @@
if (!properties.isCustom)
throw new InvalidRequestException("Only CUSTOM indexes support multiple columns");
- Set<ColumnIdentifier> columns = new HashSet<>();
+ Set<ColumnIdentifier> columns = Sets.newHashSetWithExpectedSize(targets.size());
for (IndexTarget target : targets)
if (!columns.add(target.column))
throw new InvalidRequestException("Duplicate column " + target.column + " in index target list");
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
index 6f4331b..0b0b203 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.cql3.statements;
-import java.nio.ByteBuffer;
import java.util.*;
import java.util.stream.Collectors;
@@ -25,7 +24,6 @@
import org.apache.cassandra.config.*;
import org.apache.cassandra.cql3.*;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.db.marshal.UserType;
import org.apache.cassandra.exceptions.*;
import org.apache.cassandra.schema.KeyspaceMetadata;
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 013adbc..8fe9f7e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -22,6 +22,7 @@
import java.util.stream.Collectors;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
import org.apache.cassandra.auth.Permission;
import org.apache.cassandra.config.CFMetaData;
@@ -41,7 +42,6 @@
import org.apache.cassandra.exceptions.UnauthorizedException;
import org.apache.cassandra.schema.TableParams;
import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.ClientWarn;
import org.apache.cassandra.service.MigrationManager;
import org.apache.cassandra.thrift.ThriftValidation;
import org.apache.cassandra.transport.Event;
@@ -149,7 +149,7 @@
baseName.getColumnFamily()));
}
- Set<ColumnIdentifier> included = new HashSet<>();
+ Set<ColumnIdentifier> included = Sets.newHashSetWithExpectedSize(selectClause.size());
for (RawSelector selector : selectClause)
{
Selectable.Raw selectable = selector.selectable;
@@ -203,14 +203,6 @@
if (!prepared.boundNames.isEmpty())
throw new InvalidRequestException("Cannot use query parameters in CREATE MATERIALIZED VIEW statements");
- if (!restrictions.nonPKRestrictedColumns(false).isEmpty())
- {
- throw new InvalidRequestException(String.format(
- "Non-primary key columns cannot be restricted in the SELECT statement used for materialized view " +
- "creation (got restrictions on: %s)",
- restrictions.nonPKRestrictedColumns(false).stream().map(def -> def.name.toString()).collect(Collectors.joining(", "))));
- }
-
String whereClauseText = View.relationsToWhereClause(whereClause.relations);
Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
index 1f53ac4..f9c91c6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
@@ -20,8 +20,6 @@
import org.apache.cassandra.auth.Permission;
import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.ViewDefinition;
import org.apache.cassandra.cql3.CFName;
import org.apache.cassandra.db.view.View;
import org.apache.cassandra.exceptions.ConfigurationException;
diff --git a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
index 9756a4c..84af273 100644
--- a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
+++ b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
@@ -22,7 +22,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.exceptions.InvalidRequestException;
public class IndexTarget
{
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 8d85498..675aaea 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -52,7 +52,6 @@
import org.apache.cassandra.utils.UUIDGen;
import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
-import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
import static org.apache.cassandra.cql3.statements.RequestValidations.checkNull;
/*
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index f2b484e..d3fb159 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -21,9 +21,6 @@
import java.util.*;
import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -64,7 +61,6 @@
import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
import static org.apache.cassandra.cql3.statements.RequestValidations.checkNull;
import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
-import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
import static org.apache.cassandra.utils.ByteBufferUtil.UNSET_BYTE_BUFFER;
/**
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 336091d..fa3c0f3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.cql3.statements;
-import java.io.IOException;
import java.util.concurrent.TimeoutException;
import org.apache.cassandra.auth.Permission;
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 3657f94..6bcfd9c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -203,11 +203,13 @@
public static class ParsedInsertJson extends ModificationStatement.Parsed
{
private final Json.Raw jsonValue;
+ private final boolean defaultUnset;
- public ParsedInsertJson(CFName name, Attributes.Raw attrs, Json.Raw jsonValue, boolean ifNotExists)
+ public ParsedInsertJson(CFName name, Attributes.Raw attrs, Json.Raw jsonValue, boolean defaultUnset, boolean ifNotExists)
{
super(name, StatementType.INSERT, attrs, null, ifNotExists, false);
this.jsonValue = jsonValue;
+ this.defaultUnset = defaultUnset;
}
@Override
@@ -230,7 +232,7 @@
if (def.isClusteringColumn())
hasClusteringColumnsSet = true;
- Term.Raw raw = prepared.getRawTermForColumn(def);
+ Term.Raw raw = prepared.getRawTermForColumn(def, defaultUnset);
if (def.isPrimaryKeyColumn())
{
whereClause.add(new SingleColumnRelation(ColumnDefinition.Raw.forColumn(def), Operator.EQ, raw));
diff --git a/src/java/org/apache/cassandra/db/BufferClustering.java b/src/java/org/apache/cassandra/db/BufferClustering.java
index 7c6bb20..df6a473 100644
--- a/src/java/org/apache/cassandra/db/BufferClustering.java
+++ b/src/java/org/apache/cassandra/db/BufferClustering.java
@@ -19,10 +19,6 @@
import java.nio.ByteBuffer;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
/**
* The clustering column values for a row.
* <p>
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 9d31b60..594da98 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -186,6 +186,9 @@
private static final String SAMPLING_RESULTS_NAME = "SAMPLING_RESULTS";
private static final CompositeType SAMPLING_RESULT;
+ public static final String SNAPSHOT_TRUNCATE_PREFIX = "truncated";
+ public static final String SNAPSHOT_DROP_PREFIX = "dropped";
+
static
{
try
@@ -1804,6 +1807,7 @@
}
writeSnapshotManifest(filesJSONArr, snapshotName);
+ writeSnapshotSchema(snapshotName);
}
}
if (ephemeral)
@@ -1833,6 +1837,27 @@
}
}
+ private void writeSnapshotSchema(final String snapshotName)
+ {
+ final File schemaFile = getDirectories().getSnapshotSchemaFile(snapshotName);
+
+ try
+ {
+ if (!schemaFile.getParentFile().exists())
+ schemaFile.getParentFile().mkdirs();
+
+ try (PrintStream out = new PrintStream(schemaFile))
+ {
+ for (String s: ColumnFamilyStoreCQLHelper.dumpReCreateStatements(metadata))
+ out.println(s);
+ }
+ }
+ catch (IOException e)
+ {
+ throw new FSWriteError(e, schemaFile);
+ }
+ }
+
private void createEphemeralSnapshotMarkerFile(final String snapshot)
{
final File ephemeralSnapshotMarker = getDirectories().getNewEphemeralSnapshotMarkerFile(snapshot);
@@ -2185,7 +2210,7 @@
data.notifyTruncated(truncatedAt);
if (DatabaseDescriptor.isAutoSnapshot())
- snapshot(Keyspace.getTimestampedSnapshotName(name));
+ snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(name, SNAPSHOT_TRUNCATE_PREFIX));
discardSSTables(truncatedAt);
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java
new file mode 100644
index 0000000..f040575
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java
@@ -0,0 +1,442 @@
+/*
+ * Licensed to the Apache Softw≤are 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;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+import java.util.function.*;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.statements.*;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.utils.*;
+
+/**
+ * Helper methods to represent CFMetadata and related objects in CQL format
+ */
+public class ColumnFamilyStoreCQLHelper
+{
+ public static List<String> dumpReCreateStatements(CFMetaData metadata)
+ {
+ List<String> l = new ArrayList<>();
+ // Types come first, as table can't be created without them
+ l.addAll(ColumnFamilyStoreCQLHelper.getUserTypesAsCQL(metadata));
+ // Record re-create schema statements
+ l.add(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(metadata, true));
+ // Dropped columns (and re-additions)
+ l.addAll(ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(metadata));
+ // Indexes applied as last, since otherwise they may interfere with column drops / re-additions
+ l.addAll(ColumnFamilyStoreCQLHelper.getIndexesAsCQL(metadata));
+ return l;
+ }
+
+ private static List<ColumnDefinition> getClusteringColumns(CFMetaData metadata)
+ {
+ List<ColumnDefinition> cds = new ArrayList<>(metadata.clusteringColumns().size());
+
+ if (!metadata.isStaticCompactTable())
+ for (ColumnDefinition cd : metadata.clusteringColumns())
+ cds.add(cd);
+
+ return cds;
+ }
+
+ private static List<ColumnDefinition> getPartitionColumns(CFMetaData metadata)
+ {
+ List<ColumnDefinition> cds = new ArrayList<>(metadata.partitionColumns().size());
+
+ for (ColumnDefinition cd : metadata.partitionColumns().statics)
+ cds.add(cd);
+
+ if (metadata.isDense())
+ {
+ // remove an empty type
+ for (ColumnDefinition cd : metadata.partitionColumns().withoutStatics())
+ if (!cd.type.equals(EmptyType.instance))
+ cds.add(cd);
+ }
+ // "regular" columns are not exposed for static compact tables
+ else if (!metadata.isStaticCompactTable())
+ {
+ for (ColumnDefinition cd : metadata.partitionColumns().withoutStatics())
+ cds.add(cd);
+ }
+
+ return cds;
+ }
+
+ /**
+ * Build a CQL String representation of Column Family Metadata
+ */
+ @VisibleForTesting
+ public static String getCFMetadataAsCQL(CFMetaData metadata, boolean includeDroppedColumns)
+ {
+ StringBuilder sb = new StringBuilder();
+ if (!isCqlCompatible(metadata))
+ {
+ sb.append(String.format("/*\nWarning: Table %s.%s omitted because it has constructs not compatible with CQL (was created via legacy API).\n",
+ metadata.ksName,
+ metadata.cfName));
+ sb.append("\nApproximate structure, for reference:");
+ sb.append("\n(this should not be used to reproduce this schema)\n\n");
+ }
+
+ sb.append("CREATE TABLE IF NOT EXISTS ");
+ sb.append(quoteIdentifier(metadata.ksName)).append('.').append(quoteIdentifier(metadata.cfName)).append(" (");
+
+ List<ColumnDefinition> partitionKeyColumns = metadata.partitionKeyColumns();
+ List<ColumnDefinition> clusteringColumns = getClusteringColumns(metadata);
+ List<ColumnDefinition> partitionColumns = getPartitionColumns(metadata);
+
+ Consumer<StringBuilder> cdCommaAppender = commaAppender("\n\t");
+ sb.append("\n\t");
+ for (ColumnDefinition cfd: partitionKeyColumns)
+ {
+ cdCommaAppender.accept(sb);
+ sb.append(toCQL(cfd));
+ if (partitionKeyColumns.size() == 1 && clusteringColumns.size() == 0)
+ sb.append(" PRIMARY KEY");
+ }
+
+ for (ColumnDefinition cfd: clusteringColumns)
+ {
+ cdCommaAppender.accept(sb);
+ sb.append(toCQL(cfd));
+ }
+
+ for (ColumnDefinition cfd: partitionColumns)
+ {
+ cdCommaAppender.accept(sb);
+ sb.append(toCQL(cfd, metadata.isStaticCompactTable()));
+ }
+
+ if (includeDroppedColumns)
+ {
+ for (Map.Entry<ByteBuffer, CFMetaData.DroppedColumn> entry: metadata.getDroppedColumns().entrySet())
+ {
+ if (metadata.getColumnDefinition(entry.getKey()) != null)
+ continue;
+
+ CFMetaData.DroppedColumn droppedColumn = entry.getValue();
+ cdCommaAppender.accept(sb);
+ sb.append(quoteIdentifier(droppedColumn.name));
+ sb.append(' ');
+ sb.append(droppedColumn.type.asCQL3Type().toString());
+ }
+ }
+
+ if (clusteringColumns.size() > 0 || partitionKeyColumns.size() > 1)
+ {
+ sb.append(",\n\tPRIMARY KEY (");
+ if (partitionKeyColumns.size() > 1)
+ {
+ sb.append("(");
+ Consumer<StringBuilder> pkCommaAppender = commaAppender(" ");
+ for (ColumnDefinition cfd : partitionKeyColumns)
+ {
+ pkCommaAppender.accept(sb);
+ sb.append(quoteIdentifier(cfd.name.toString()));
+ }
+ sb.append(")");
+ }
+ else
+ {
+ sb.append(quoteIdentifier(partitionKeyColumns.get(0).name.toString()));
+ }
+
+ for (ColumnDefinition cfd : metadata.clusteringColumns())
+ sb.append(", ").append(quoteIdentifier(cfd.name.toString()));
+
+ sb.append(')');
+ }
+ sb.append(")\n\t");
+ sb.append("WITH ");
+
+ sb.append("ID = ").append(metadata.cfId).append("\n\tAND ");
+
+ if (metadata.isCompactTable())
+ sb.append("COMPACT STORAGE\n\tAND ");
+
+ if (clusteringColumns.size() > 0)
+ {
+ sb.append("CLUSTERING ORDER BY (");
+
+ Consumer<StringBuilder> cOrderCommaAppender = commaAppender(" ");
+ for (ColumnDefinition cd : clusteringColumns)
+ {
+ cOrderCommaAppender.accept(sb);
+ sb.append(quoteIdentifier(cd.name.toString())).append(' ').append(cd.clusteringOrder().toString());
+ }
+ sb.append(")\n\tAND ");
+ }
+
+ sb.append(toCQL(metadata.params));
+ sb.append(";");
+
+ if (!isCqlCompatible(metadata))
+ {
+ sb.append("\n*/");
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Build a CQL String representation of User Types used in the given Column Family.
+ *
+ * Type order is ensured as types are built incrementally: from the innermost (most nested)
+ * to the outermost.
+ */
+ @VisibleForTesting
+ public static List<String> getUserTypesAsCQL(CFMetaData metadata)
+ {
+ List<AbstractType> types = new ArrayList<>();
+ Set<AbstractType> typeSet = new HashSet<>();
+ for (ColumnDefinition cd: Iterables.concat(metadata.partitionKeyColumns(), metadata.clusteringColumns(), metadata.partitionColumns()))
+ {
+ AbstractType type = cd.type;
+ if (type.isUDT())
+ resolveUserType((UserType) type, typeSet, types);
+ }
+
+ List<String> typeStrings = new ArrayList<>();
+ for (AbstractType type: types)
+ typeStrings.add(toCQL((UserType) type));
+ return typeStrings;
+ }
+
+ /**
+ * Build a CQL String representation of Dropped Columns in the given Column Family.
+ *
+ * If the column was dropped once, but is now re-created `ADD` will be appended accordingly.
+ */
+ @VisibleForTesting
+ public static List<String> getDroppedColumnsAsCQL(CFMetaData metadata)
+ {
+ List<String> droppedColumns = new ArrayList<>();
+
+ for (Map.Entry<ByteBuffer, CFMetaData.DroppedColumn> entry: metadata.getDroppedColumns().entrySet())
+ {
+ CFMetaData.DroppedColumn column = entry.getValue();
+ droppedColumns.add(toCQLDrop(metadata.ksName, metadata.cfName, column));
+ if (metadata.getColumnDefinition(entry.getKey()) != null)
+ droppedColumns.add(toCQLAdd(metadata.ksName, metadata.cfName, metadata.getColumnDefinition(entry.getKey())));
+ }
+
+ return droppedColumns;
+ }
+
+ /**
+ * Build a CQL String representation of Indexes on columns in the given Column Family
+ */
+ @VisibleForTesting
+ public static List<String> getIndexesAsCQL(CFMetaData metadata)
+ {
+ List<String> indexes = new ArrayList<>();
+ for (IndexMetadata indexMetadata: metadata.getIndexes())
+ indexes.add(toCQL(metadata.ksName, metadata.cfName, indexMetadata));
+ return indexes;
+ }
+
+ private static String toCQL(String keyspace, String cf, IndexMetadata indexMetadata)
+ {
+ if (indexMetadata.isCustom())
+ {
+ Map<String, String> options = new HashMap<>();
+ indexMetadata.options.forEach((k, v) -> {
+ if (!k.equals(IndexTarget.TARGET_OPTION_NAME) && !k.equals(IndexTarget.CUSTOM_INDEX_OPTION_NAME))
+ options.put(k, v);
+ });
+
+ return String.format("CREATE CUSTOM INDEX %s ON %s.%s (%s) USING '%s'%s;",
+ quoteIdentifier(indexMetadata.name),
+ quoteIdentifier(keyspace),
+ quoteIdentifier(cf),
+ indexMetadata.options.get(IndexTarget.TARGET_OPTION_NAME),
+ indexMetadata.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME),
+ options.isEmpty() ? "" : " WITH OPTIONS " + toCQL(options));
+ }
+ else
+ {
+ return String.format("CREATE INDEX %s ON %s.%s (%s);",
+ quoteIdentifier(indexMetadata.name),
+ quoteIdentifier(keyspace),
+ quoteIdentifier(cf),
+ indexMetadata.options.get(IndexTarget.TARGET_OPTION_NAME));
+ }
+ }
+ private static String toCQL(UserType userType)
+ {
+ StringBuilder sb = new StringBuilder();
+ sb.append(String.format("CREATE TYPE %s.%s(",
+ quoteIdentifier(userType.keyspace),
+ quoteIdentifier(userType.getNameAsString())));
+
+ Consumer<StringBuilder> commaAppender = commaAppender(" ");
+ for (int i = 0; i < userType.size(); i++)
+ {
+ commaAppender.accept(sb);
+ sb.append(String.format("%s %s",
+ userType.fieldNameAsString(i),
+ userType.fieldType(i).asCQL3Type()));
+ }
+ sb.append(");");
+ return sb.toString();
+ }
+
+ private static String toCQL(TableParams tableParams)
+ {
+ StringBuilder builder = new StringBuilder();
+
+ builder.append("bloom_filter_fp_chance = ").append(tableParams.bloomFilterFpChance);
+ builder.append("\n\tAND dclocal_read_repair_chance = ").append(tableParams.dcLocalReadRepairChance);
+ builder.append("\n\tAND crc_check_chance = ").append(tableParams.crcCheckChance);
+ builder.append("\n\tAND default_time_to_live = ").append(tableParams.defaultTimeToLive);
+ builder.append("\n\tAND gc_grace_seconds = ").append(tableParams.gcGraceSeconds);
+ builder.append("\n\tAND min_index_interval = ").append(tableParams.minIndexInterval);
+ builder.append("\n\tAND max_index_interval = ").append(tableParams.maxIndexInterval);
+ builder.append("\n\tAND memtable_flush_period_in_ms = ").append(tableParams.memtableFlushPeriodInMs);
+ builder.append("\n\tAND read_repair_chance = ").append(tableParams.readRepairChance);
+ builder.append("\n\tAND speculative_retry = '").append(tableParams.speculativeRetry).append("'");
+ builder.append("\n\tAND comment = ").append(singleQuote(tableParams.comment));
+ builder.append("\n\tAND caching = ").append(toCQL(tableParams.caching.asMap()));
+ builder.append("\n\tAND compaction = ").append(toCQL(tableParams.compaction.asMap()));
+ builder.append("\n\tAND compression = ").append(toCQL(tableParams.compression.asMap()));
+ builder.append("\n\tAND cdc = ").append(tableParams.cdc);
+
+ builder.append("\n\tAND extensions = { ");
+ for (Map.Entry<String, ByteBuffer> entry : tableParams.extensions.entrySet())
+ {
+ builder.append(singleQuote(entry.getKey()));
+ builder.append(": ");
+ builder.append("0x" + ByteBufferUtil.bytesToHex(entry.getValue()));
+ }
+ builder.append(" }");
+ return builder.toString();
+ }
+
+ private static String toCQL(Map<?, ?> map)
+ {
+ StringBuilder builder = new StringBuilder("{ ");
+
+ boolean isFirst = true;
+ for (Map.Entry entry: map.entrySet())
+ {
+ if (isFirst)
+ isFirst = false;
+ else
+ builder.append(", ");
+ builder.append(singleQuote(entry.getKey().toString()));
+ builder.append(": ");
+ builder.append(singleQuote(entry.getValue().toString()));
+ }
+
+ builder.append(" }");
+ return builder.toString();
+ }
+
+ private static String toCQL(ColumnDefinition cd)
+ {
+ return toCQL(cd, false);
+ }
+
+ private static String toCQL(ColumnDefinition cd, boolean isStaticCompactTable)
+ {
+ return String.format("%s %s%s",
+ quoteIdentifier(cd.name.toString()),
+ cd.type.asCQL3Type().toString(),
+ cd.isStatic() && !isStaticCompactTable ? " static" : "");
+ }
+
+ private static String toCQLAdd(String keyspace, String cf, ColumnDefinition cd)
+ {
+ return String.format("ALTER TABLE %s.%s ADD %s %s%s;",
+ quoteIdentifier(keyspace),
+ quoteIdentifier(cf),
+ quoteIdentifier(cd.name.toString()),
+ cd.type.asCQL3Type().toString(),
+ cd.isStatic() ? " static" : "");
+ }
+
+ private static String toCQLDrop(String keyspace, String cf, CFMetaData.DroppedColumn droppedColumn)
+ {
+ return String.format("ALTER TABLE %s.%s DROP %s USING TIMESTAMP %s;",
+ quoteIdentifier(keyspace),
+ quoteIdentifier(cf),
+ quoteIdentifier(droppedColumn.name),
+ droppedColumn.droppedTime);
+ }
+
+ private static void resolveUserType(UserType type, Set<AbstractType> typeSet, List<AbstractType> types)
+ {
+ for (AbstractType subType: type.fieldTypes())
+ if (!typeSet.contains(subType) && subType.isUDT())
+ resolveUserType((UserType) subType, typeSet, types);
+
+ if (!typeSet.contains(type))
+ {
+ typeSet.add(type);
+ types.add(type);
+ }
+ }
+
+ private static String singleQuote(String s)
+ {
+ return String.format("'%s'", s.replaceAll("'", "''"));
+ }
+
+ private static Consumer<StringBuilder> commaAppender(String afterComma)
+ {
+ AtomicBoolean isFirst = new AtomicBoolean(true);
+ return new Consumer<StringBuilder>()
+ {
+ public void accept(StringBuilder stringBuilder)
+ {
+ if (!isFirst.getAndSet(false))
+ stringBuilder.append(',').append(afterComma);
+ }
+ };
+ }
+
+ private static String quoteIdentifier(String id)
+ {
+ return ColumnIdentifier.maybeQuote(id);
+ }
+
+ /**
+ * Whether or not the given metadata is compatible / representable with CQL Language
+ */
+ public static boolean isCqlCompatible(CFMetaData metaData)
+ {
+ if (metaData.isSuper())
+ return false;
+
+ if (metaData.isCompactTable()
+ && metaData.partitionColumns().withoutStatics().size() > 1
+ && metaData.clusteringColumns().size() >= 1)
+ return false;
+
+ return true;
+ }
+}
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java b/src/java/org/apache/cassandra/db/ColumnIndex.java
index 2e7a2ee..9cea084 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -48,6 +48,8 @@
// used, until the row-index-entry reaches config column_index_cache_size_in_kb
private final List<IndexInfo> indexSamples = new ArrayList<>();
+ private DataOutputBuffer reusableBuffer;
+
public int columnIndexCount;
private int[] indexOffsets;
@@ -95,6 +97,8 @@
this.firstClustering = null;
this.lastClustering = null;
this.openMarker = null;
+ if (this.buffer != null)
+ this.reusableBuffer = this.buffer;
this.buffer = null;
}
@@ -195,7 +199,7 @@
indexSamplesSerializedSize += idxSerializer.serializedSize(cIndexInfo);
if (indexSamplesSerializedSize + columnIndexCount * TypeSizes.sizeof(0) > DatabaseDescriptor.getColumnIndexCacheSize())
{
- buffer = new DataOutputBuffer(DatabaseDescriptor.getColumnIndexCacheSize() * 2);
+ buffer = useBuffer();
for (IndexInfo indexSample : indexSamples)
{
idxSerializer.serialize(indexSample, buffer);
@@ -215,6 +219,16 @@
firstClustering = null;
}
+ private DataOutputBuffer useBuffer()
+ {
+ if (reusableBuffer != null) {
+ buffer = reusableBuffer;
+ buffer.clear();
+ }
+ // don't use the standard RECYCLER as that only recycles up to 1MB and requires proper cleanup
+ return new DataOutputBuffer(DatabaseDescriptor.getColumnIndexCacheSize() * 2);
+ }
+
private void add(Unfiltered unfiltered) throws IOException
{
long pos = currentPosition();
diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java
index e3c30fa..e9e3abf 100644
--- a/src/java/org/apache/cassandra/db/Columns.java
+++ b/src/java/org/apache/cassandra/db/Columns.java
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.util.*;
+import java.util.function.Consumer;
import java.util.function.Predicate;
import java.nio.ByteBuffer;
import java.security.MessageDigest;
@@ -366,6 +367,16 @@
digest.update(c.name.bytes.duplicate());
}
+ /**
+ * Apply a function to each column definition in forwards or reversed order.
+ * @param function
+ * @param reversed
+ */
+ public void apply(Consumer<ColumnDefinition> function, boolean reversed)
+ {
+ BTree.apply(columns, function, reversed);
+ }
+
@Override
public boolean equals(Object other)
{
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index 6a07782..4e4a30d 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -43,7 +43,6 @@
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.concurrent.OpOrder;
import org.apache.cassandra.utils.btree.BTreeSet;
public class CounterMutation implements IMutation
diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
index f89480a..d1b1fa2 100644
--- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
@@ -21,7 +21,6 @@
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.RequestExecutionException;
import org.apache.cassandra.net.IVerbHandler;
import org.apache.cassandra.net.MessageIn;
import org.apache.cassandra.net.MessagingService;
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index cd5b695..62fd890 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -17,8 +17,6 @@
*/
package org.apache.cassandra.db;
-import static com.google.common.collect.Sets.newHashSet;
-
import java.io.File;
import java.io.FileFilter;
import java.io.IOError;
@@ -508,6 +506,12 @@
return new File(snapshotDir, "manifest.json");
}
+ public File getSnapshotSchemaFile(String snapshotName)
+ {
+ File snapshotDir = getSnapshotDirectory(getDirectoryForNewSSTables(), snapshotName);
+ return new File(snapshotDir, "schema.cql");
+ }
+
public File getNewEphemeralSnapshotMarkerFile(String snapshotName)
{
File snapshotDir = new File(getWriteableLocationAsFile(1L), join(SNAPSHOT_SUBDIR, snapshotName));
@@ -921,7 +925,7 @@
if (!input.isDirectory())
return 0;
- SSTableSizeSummer visitor = new SSTableSizeSummer(sstableLister(Directories.OnTxnErr.THROW).listFiles());
+ SSTableSizeSummer visitor = new SSTableSizeSummer(input, sstableLister(Directories.OnTxnErr.THROW).listFiles());
try
{
Files.walkFileTree(input.toPath(), visitor);
@@ -1007,21 +1011,22 @@
private class SSTableSizeSummer extends DirectorySizeCalculator
{
- SSTableSizeSummer(List<File> files)
+ private final HashSet<File> toSkip;
+ SSTableSizeSummer(File path, List<File> files)
{
- super(files);
+ super(path);
+ toSkip = new HashSet<>(files);
}
@Override
- public boolean isAcceptable(Path file)
+ public boolean isAcceptable(Path path)
{
- String fileName = file.toFile().getName();
- Pair<Descriptor, Component> pair = SSTable.tryComponentFromFilename(file.getParent().toFile(), fileName);
+ File file = path.toFile();
+ Pair<Descriptor, Component> pair = SSTable.tryComponentFromFilename(path.getParent().toFile(), file.getName());
return pair != null
&& pair.left.ksname.equals(metadata.ksName)
&& pair.left.cfname.equals(metadata.cfName)
- && !visited.contains(fileName)
- && !alive.contains(fileName);
+ && !toSkip.contains(file);
}
}
}
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 6e44308..37ed1e1 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -263,6 +263,11 @@
return snapshotName;
}
+ public static String getTimestampedSnapshotNameWithPrefix(String clientSuppliedName, String prefix)
+ {
+ return prefix + "-" + getTimestampedSnapshotName(clientSuppliedName);
+ }
+
/**
* Check whether snapshots already exists for a given name.
*
diff --git a/src/java/org/apache/cassandra/db/LivenessInfo.java b/src/java/org/apache/cassandra/db/LivenessInfo.java
index 71f3101..e6e336b 100644
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@ -20,7 +20,6 @@
import java.util.Objects;
import java.security.MessageDigest;
-import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.serializers.MarshalException;
import org.apache.cassandra.utils.FBUtilities;
diff --git a/src/java/org/apache/cassandra/db/MutableDeletionInfo.java b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
index 4a2c455..39728c5 100644
--- a/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/MutableDeletionInfo.java
@@ -21,7 +21,6 @@
import java.util.Iterator;
import com.google.common.base.Objects;
-import com.google.common.collect.Iterators;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.rows.EncodingStats;
diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
index 74dd625..5888438 100644
--- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
@@ -20,7 +20,6 @@
import java.io.DataInputStream;
import java.io.IOException;
import java.net.InetAddress;
-import java.util.function.Consumer;
import org.apache.cassandra.batchlog.LegacyBatchlogMigrator;
import org.apache.cassandra.exceptions.WriteTimeoutException;
diff --git a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
index 019209e..add5218 100644
--- a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
@@ -22,7 +22,6 @@
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.HeapAllocator;
import org.apache.cassandra.utils.memory.MemoryUtil;
import org.apache.cassandra.utils.memory.NativeAllocator;
diff --git a/src/java/org/apache/cassandra/db/PartitionPosition.java b/src/java/org/apache/cassandra/db/PartitionPosition.java
index ac5258d..3b45c6c 100644
--- a/src/java/org/apache/cassandra/db/PartitionPosition.java
+++ b/src/java/org/apache/cassandra/db/PartitionPosition.java
@@ -23,7 +23,6 @@
import org.apache.cassandra.dht.*;
import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.ByteBufferUtil;
public interface PartitionPosition extends RingPosition<PartitionPosition>
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index 9517503..b13bf39 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -27,7 +27,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.lifecycle.SSTableSet;
import org.apache.cassandra.db.lifecycle.View;
import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.db.rows.BaseRowIterator;
diff --git a/src/java/org/apache/cassandra/db/RangeTombstoneList.java b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
index c60b774..716213d 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstoneList.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
@@ -549,7 +549,7 @@
ClusteringBound newEnd = start.invert();
if (!Slice.isEmpty(comparator, starts[i], newEnd))
{
- addInternal(i, starts[i], start.invert(), markedAts[i], delTimes[i]);
+ addInternal(i, starts[i], newEnd, markedAts[i], delTimes[i]);
i++;
setInternal(i, start, ends[i], markedAts[i], delTimes[i]);
}
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 68c9e3b..372cf1c 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -59,7 +59,7 @@
*/
public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
{
- private static final int TEST_ITERATION_DELAY_MILLIS = Integer.valueOf(System.getProperty("cassandra.test.read_iteration_delay_ms", "0"));
+ private static final int TEST_ITERATION_DELAY_MILLIS = Integer.parseInt(System.getProperty("cassandra.test.read_iteration_delay_ms", "0"));
protected static final Logger logger = LoggerFactory.getLogger(ReadCommand.class);
public static final IVersionedSerializer<ReadCommand> serializer = new Serializer();
diff --git a/src/java/org/apache/cassandra/db/RowIndexEntry.java b/src/java/org/apache/cassandra/db/RowIndexEntry.java
index dd1fdb7..d030329 100644
--- a/src/java/org/apache/cassandra/db/RowIndexEntry.java
+++ b/src/java/org/apache/cassandra/db/RowIndexEntry.java
@@ -33,7 +33,7 @@
import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.FileHandle;
import org.apache.cassandra.io.util.TrackedDataInputPlus;
import org.apache.cassandra.metrics.DefaultNameFactory;
import org.apache.cassandra.metrics.MetricNameFactory;
@@ -229,7 +229,7 @@
return new RowIndexEntry<>(dataFilePosition);
}
- public IndexInfoRetriever openWithIndex(SegmentedFile indexFile)
+ public IndexInfoRetriever openWithIndex(FileHandle indexFile)
{
return null;
}
@@ -480,7 +480,7 @@
}
@Override
- public IndexInfoRetriever openWithIndex(SegmentedFile indexFile)
+ public IndexInfoRetriever openWithIndex(FileHandle indexFile)
{
int fieldsSize = (int) DeletionTime.serializer.serializedSize(deletionTime)
+ TypeSizes.sizeof(0); // columnIndexCount
@@ -715,7 +715,7 @@
}
@Override
- public IndexInfoRetriever openWithIndex(SegmentedFile indexFile)
+ public IndexInfoRetriever openWithIndex(FileHandle indexFile)
{
indexEntrySizeHistogram.update(serializedSize(deletionTime, headerLength, columnsIndex.length) + indexedPartSize);
indexInfoCountHistogram.update(columnsIndex.length);
@@ -879,7 +879,7 @@
}
@Override
- public IndexInfoRetriever openWithIndex(SegmentedFile indexFile)
+ public IndexInfoRetriever openWithIndex(FileHandle indexFile)
{
indexEntrySizeHistogram.update(indexedPartSize + fieldsSerializedSize);
indexInfoCountHistogram.update(columnsIndexCount);
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index af2d434..9c443c7 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -21,9 +21,6 @@
import java.nio.ByteBuffer;
import java.util.*;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.db.filter.ColumnFilter;
@@ -108,17 +105,12 @@
{
this(isForSSTable,
metadata.getKeyValidator(),
- typesOf(metadata.clusteringColumns()),
+ metadata.comparator.subtypes(),
columns,
stats,
null);
}
- private static List<AbstractType<?>> typesOf(List<ColumnDefinition> columns)
- {
- return ImmutableList.copyOf(Lists.transform(columns, column -> column.type));
- }
-
public PartitionColumns columns()
{
return columns;
@@ -398,7 +390,7 @@
EncodingStats stats = EncodingStats.serializer.deserialize(in);
AbstractType<?> keyType = metadata.getKeyValidator();
- List<AbstractType<?>> clusteringTypes = typesOf(metadata.clusteringColumns());
+ List<AbstractType<?>> clusteringTypes = metadata.comparator.subtypes();
Columns statics, regulars;
if (selection == null)
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 855b030..d96648f 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -54,7 +54,6 @@
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.SearchIterator;
import org.apache.cassandra.utils.btree.BTreeSet;
-import org.apache.cassandra.utils.concurrent.OpOrder;
/**
diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java
index f880781..b3fd20a 100644
--- a/src/java/org/apache/cassandra/db/Slices.java
+++ b/src/java/org/apache/cassandra/db/Slices.java
@@ -25,7 +25,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 584279d..6451c83 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -97,6 +97,7 @@
public static final String AVAILABLE_RANGES = "available_ranges";
public static final String VIEWS_BUILDS_IN_PROGRESS = "views_builds_in_progress";
public static final String BUILT_VIEWS = "built_views";
+ public static final String PREPARED_STATEMENTS = "prepared_statements";
@Deprecated public static final String LEGACY_HINTS = "hints";
@Deprecated public static final String LEGACY_BATCHLOG = "batchlog";
@@ -265,6 +266,15 @@
+ "status_replicated boolean,"
+ "PRIMARY KEY ((keyspace_name), view_name))");
+ private static final CFMetaData PreparedStatements =
+ compile(PREPARED_STATEMENTS,
+ "prepared statements",
+ "CREATE TABLE %s ("
+ + "prepared_id blob,"
+ + "logged_keyspace text,"
+ + "query_string text,"
+ + "PRIMARY KEY ((prepared_id)))");
+
@Deprecated
public static final CFMetaData LegacyHints =
compile(LEGACY_HINTS,
@@ -435,6 +445,7 @@
BuiltViews,
LegacyHints,
LegacyBatchlog,
+ PreparedStatements,
LegacyKeyspaces,
LegacyColumnfamilies,
LegacyColumns,
@@ -641,7 +652,7 @@
private static Map<UUID, ByteBuffer> truncationAsMapEntry(ColumnFamilyStore cfs, long truncatedAt, CommitLogPosition position)
{
DataOutputBuffer out = null;
- try (DataOutputBuffer ignored = out = DataOutputBuffer.RECYCLER.get())
+ try (DataOutputBuffer ignored = out = DataOutputBuffer.scratchBuffer.get())
{
CommitLogPosition.serializer.serialize(position, out);
out.writeLong(truncatedAt);
@@ -1412,4 +1423,31 @@
}
}
+ public static void writePreparedStatement(String loggedKeyspace, MD5Digest key, String cql)
+ {
+ executeInternal(String.format("INSERT INTO %s.%s"
+ + " (logged_keyspace, prepared_id, query_string) VALUES (?, ?, ?)",
+ NAME, PREPARED_STATEMENTS),
+ loggedKeyspace, key.byteBuffer(), cql);
+ logger.debug("stored prepared statement for logged keyspace '{}': '{}'", loggedKeyspace, cql);
+ }
+
+ public static void removePreparedStatement(MD5Digest key)
+ {
+ executeInternal(String.format("DELETE FROM %s.%s"
+ + " WHERE prepared_id = ?",
+ NAME, PREPARED_STATEMENTS),
+ key.byteBuffer());
+ }
+
+ public static List<Pair<String, String>> loadPreparedStatements()
+ {
+ String query = String.format("SELECT logged_keyspace, query_string FROM %s.%s", NAME, PREPARED_STATEMENTS);
+ UntypedResultSet resultSet = executeOnceInternal(query);
+ List<Pair<String, String>> r = new ArrayList<>();
+ for (UntypedResultSet.Row row : resultSet)
+ r.add(Pair.create(row.has("logged_keyspace") ? row.getString("logged_keyspace") : null,
+ row.getString("query_string")));
+ return r;
+ }
}
diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index 005bb2c..16aa30a 100644
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@ -31,7 +31,7 @@
import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.DataPosition;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.FileHandle;
import org.apache.cassandra.utils.ByteBufferUtil;
public abstract class AbstractSSTableIterator implements UnfilteredRowIterator
@@ -47,7 +47,7 @@
private final boolean isForThrift;
- protected final SegmentedFile ifile;
+ protected final FileHandle ifile;
private boolean isClosed;
@@ -63,7 +63,7 @@
Slices slices,
ColumnFilter columnFilter,
boolean isForThrift,
- SegmentedFile ifile)
+ FileHandle ifile)
{
this.sstable = sstable;
this.ifile = ifile;
@@ -453,7 +453,7 @@
// Marks the beginning of the block corresponding to currentIndexIdx.
private DataPosition mark;
- public IndexState(Reader reader, ClusteringComparator comparator, RowIndexEntry indexEntry, boolean reversed, SegmentedFile indexFile)
+ public IndexState(Reader reader, ClusteringComparator comparator, RowIndexEntry indexEntry, boolean reversed, FileHandle indexFile)
{
this.reader = reader;
this.comparator = comparator;
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
index e4f6700..ad3732f 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
@@ -25,7 +25,7 @@
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.FileHandle;
/**
* A Cell Iterator over SSTable
@@ -39,7 +39,7 @@
Slices slices,
ColumnFilter columns,
boolean isForThrift,
- SegmentedFile ifile)
+ FileHandle ifile)
{
super(sstable, file, key, indexEntry, slices, columns, isForThrift, ifile);
}
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 903fd59..8f576f1 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -27,7 +27,7 @@
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.FileHandle;
import org.apache.cassandra.utils.btree.BTree;
/**
@@ -42,7 +42,7 @@
Slices slices,
ColumnFilter columns,
boolean isForThrift,
- SegmentedFile ifile)
+ FileHandle ifile)
{
super(sstable, file, key, indexEntry, slices, columns, isForThrift, ifile);
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
index 7ea7439..275d5b3 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@ -22,16 +22,18 @@
import java.util.*;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.LockSupport;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import net.nicoulaj.compilecommand.annotations.DontInline;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.*;
+import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.concurrent.WaitQueue;
@@ -45,19 +47,27 @@
{
static final Logger logger = LoggerFactory.getLogger(AbstractCommitLogSegmentManager.class);
- // Queue of work to be done by the manager thread, also used to wake the thread to perform segment allocation.
- private final BlockingQueue<Runnable> segmentManagementTasks = new LinkedBlockingQueue<>();
+ /**
+ * Segment that is ready to be used. The management thread fills this and blocks until consumed.
+ *
+ * A single management thread produces this, and consumers are already synchronizing to make sure other work is
+ * performed atomically with consuming this. Volatile to make sure writes by the management thread become
+ * visible (ordered/lazySet would suffice). Consumers (advanceAllocatingFrom and discardAvailableSegment) must
+ * synchronize on 'this'.
+ */
+ private volatile CommitLogSegment availableSegment = null;
- /** Segments that are ready to be used. Head of the queue is the one we allocate writes to */
- private final ConcurrentLinkedQueue<CommitLogSegment> availableSegments = new ConcurrentLinkedQueue<>();
+ private final WaitQueue segmentPrepared = new WaitQueue();
- /** Active segments, containing unflushed data */
+ /** Active segments, containing unflushed data. The tail of this queue is the one we allocate writes to */
private final ConcurrentLinkedQueue<CommitLogSegment> activeSegments = new ConcurrentLinkedQueue<>();
- /** The segment we are currently allocating commit log records to */
- protected volatile CommitLogSegment allocatingFrom = null;
-
- private final WaitQueue hasAvailableSegments = new WaitQueue();
+ /**
+ * The segment we are currently allocating commit log records to.
+ *
+ * Written by advanceAllocatingFrom which synchronizes on 'this'. Volatile to ensure reads get current value.
+ */
+ private volatile CommitLogSegment allocatingFrom = null;
final String storageDirectory;
@@ -69,15 +79,9 @@
*/
private final AtomicLong size = new AtomicLong();
- /**
- * New segment creation is initially disabled because we'll typically get some "free" segments
- * recycled after log replay.
- */
- volatile boolean createReserveSegments = false;
-
private Thread managerThread;
- protected volatile boolean run = true;
protected final CommitLog commitLog;
+ private volatile boolean shutdown;
private static final SimpleCachedBufferPool bufferPool =
new SimpleCachedBufferPool(DatabaseDescriptor.getCommitLogMaxCompressionBuffersInPool(), DatabaseDescriptor.getCommitLogSegmentSize());
@@ -95,54 +99,33 @@
{
public void runMayThrow() throws Exception
{
- while (run)
+ while (!shutdown)
{
try
{
- Runnable task = segmentManagementTasks.poll();
- if (task == null)
+ assert availableSegment == null;
+ logger.debug("No segments in reserve; creating a fresh one");
+ availableSegment = createSegment();
+ if (shutdown)
{
- // if we have no more work to do, check if we should create a new segment
- if (!atSegmentLimit() &&
- availableSegments.isEmpty() &&
- (activeSegments.isEmpty() || createReserveSegments))
- {
- logger.trace("No segments in reserve; creating a fresh one");
- // TODO : some error handling in case we fail to create a new segment
- availableSegments.add(createSegment());
- hasAvailableSegments.signalAll();
- }
-
- // flush old Cfs if we're full
- long unused = unusedCapacity();
- if (unused < 0)
- {
- List<CommitLogSegment> segmentsToRecycle = new ArrayList<>();
- long spaceToReclaim = 0;
- for (CommitLogSegment segment : activeSegments)
- {
- if (segment == allocatingFrom)
- break;
- segmentsToRecycle.add(segment);
- spaceToReclaim += DatabaseDescriptor.getCommitLogSegmentSize();
- if (spaceToReclaim + unused >= 0)
- break;
- }
- flushDataFrom(segmentsToRecycle, false);
- }
-
- // Since we're operating on a "null" allocation task, block here for the next task on the
- // queue rather than looping, grabbing another null, and repeating the above work.
- try
- {
- task = segmentManagementTasks.take();
- }
- catch (InterruptedException e)
- {
- throw new AssertionError();
- }
+ // If shutdown() started and finished during segment creation, we are now left with a
+ // segment that no one will consume. Discard it.
+ discardAvailableSegment();
+ return;
}
- task.run();
+
+ segmentPrepared.signalAll();
+ Thread.yield();
+
+ if (availableSegment == null && !atSegmentBufferLimit())
+ // Writing threads need another segment now.
+ continue;
+
+ // Writing threads are not waiting for new segments, we can spend time on other tasks.
+ // flush old Cfs if we're full
+ maybeFlushToReclaim();
+
+ LockSupport.park();
}
catch (Throwable t)
{
@@ -151,27 +134,51 @@
return;
// sleep some arbitrary period to avoid spamming CL
Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
- }
- }
- }
- private boolean atSegmentLimit()
- {
- return CommitLogSegment.usesBufferPool(commitLog) && bufferPool.atLimit();
+ // If we offered a segment, wait for it to be taken before reentering the loop.
+ // There could be a new segment in next not offered, but only on failure to discard it while
+ // shutting down-- nothing more can or needs to be done in that case.
+ }
+
+ while (availableSegment != null || atSegmentBufferLimit() && !shutdown)
+ LockSupport.park();
+ }
}
};
- run = true;
-
+ shutdown = false;
managerThread = new Thread(runnable, "COMMIT-LOG-ALLOCATOR");
managerThread.start();
+
+ // for simplicity, ensure the first segment is allocated before continuing
+ advanceAllocatingFrom(null);
}
+ private boolean atSegmentBufferLimit()
+ {
+ return CommitLogSegment.usesBufferPool(commitLog) && bufferPool.atLimit();
+ }
- /**
- * Shut down the CLSM. Used both during testing and during regular shutdown, so needs to stop everything.
- */
- public abstract void shutdown();
+ private void maybeFlushToReclaim()
+ {
+ long unused = unusedCapacity();
+ if (unused < 0)
+ {
+ long flushingSize = 0;
+ List<CommitLogSegment> segmentsToRecycle = new ArrayList<>();
+ for (CommitLogSegment segment : activeSegments)
+ {
+ if (segment == allocatingFrom)
+ break;
+ flushingSize += segment.onDiskSize();
+ segmentsToRecycle.add(segment);
+ if (flushingSize + unused >= 0)
+ break;
+ }
+ flushDataFrom(segmentsToRecycle, false);
+ }
+ }
+
/**
* Allocate a segment within this CLSM. Should either succeed or throw.
@@ -200,102 +207,69 @@
*/
abstract void discard(CommitLogSegment segment, boolean delete);
-
/**
- * Grab the current CommitLogSegment we're allocating from. Also serves as a utility method to block while the allocator
- * is working on initial allocation of a CommitLogSegment.
- */
- CommitLogSegment allocatingFrom()
- {
- CommitLogSegment r = allocatingFrom;
- if (r == null)
- {
- advanceAllocatingFrom(null);
- r = allocatingFrom;
- }
- return r;
- }
-
- /**
- * Fetches a new segment from the queue, signaling the management thread to create a new one if necessary, and "activates" it.
- * Blocks until a new segment is allocated and the thread requesting an advanceAllocatingFrom is signalled.
+ * Advances the allocatingFrom pointer to the next prepared segment, but only if it is currently the segment provided.
*
* WARNING: Assumes segment management thread always succeeds in allocating a new segment or kills the JVM.
*/
- protected void advanceAllocatingFrom(CommitLogSegment old)
+ @DontInline
+ void advanceAllocatingFrom(CommitLogSegment old)
{
- while (true)
- {
- CommitLogSegment next;
+ while (true) {
synchronized (this)
{
- // do this in a critical section so we can atomically remove from availableSegments and add to allocatingFrom/activeSegments
- // see https://issues.apache.org/jira/browse/CASSANDRA-6557?focusedCommentId=13874432&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13874432
+ // do this in a critical section so we can maintain the order of segment construction when moving to allocatingFrom/activeSegments
if (allocatingFrom != old)
return;
- next = availableSegments.poll();
- if (next != null)
+
+ // If a segment is ready, take it now, otherwise wait for the management thread to construct it.
+ if (availableSegment != null)
{
- allocatingFrom = next;
- activeSegments.add(next);
+ // Success! Change allocatingFrom and activeSegments (which must be kept in order) before leaving
+ // the critical section.
+ activeSegments.add(allocatingFrom = availableSegment);
+ availableSegment = null;
+ break;
}
}
- if (next != null)
- {
- if (old != null)
- {
- // Now we can run the user defined command just after switching to the new commit log.
- // (Do this here instead of in the recycle call so we can get a head start on the archive.)
- commitLog.archiver.maybeArchive(old);
-
- // ensure we don't continue to use the old file; not strictly necessary, but cleaner to enforce it
- old.discardUnusedTail();
- }
-
- // request that the CL be synced out-of-band, as we've finished a segment
- commitLog.requestExtraSync();
- return;
- }
-
- // no more segments, so register to receive a signal when not empty
- WaitQueue.Signal signal = hasAvailableSegments.register(commitLog.metrics.waitingOnSegmentAllocation.time());
-
- // trigger the management thread; this must occur after registering
- // the signal to ensure we are woken by any new segment creation
- wakeManager();
-
- // check if the queue has already been added to before waiting on the signal, to catch modifications
- // that happened prior to registering the signal; *then* check to see if we've been beaten to making the change
- if (!availableSegments.isEmpty() || allocatingFrom != old)
- {
- signal.cancel();
- // if we've been beaten, just stop immediately
- if (allocatingFrom != old)
- return;
- // otherwise try again, as there should be an available segment
- continue;
- }
-
- // can only reach here if the queue hasn't been inserted into
- // before we registered the signal, as we only remove items from the queue
- // after updating allocatingFrom. Can safely block until we are signalled
- // by the allocator that new segments have been published
- signal.awaitUninterruptibly();
+ awaitAvailableSegment(old);
}
+
+ // Signal the management thread to prepare a new segment.
+ wakeManager();
+
+ if (old != null)
+ {
+ // Now we can run the user defined command just after switching to the new commit log.
+ // (Do this here instead of in the recycle call so we can get a head start on the archive.)
+ commitLog.archiver.maybeArchive(old);
+
+ // ensure we don't continue to use the old file; not strictly necessary, but cleaner to enforce it
+ old.discardUnusedTail();
+ }
+
+ // request that the CL be synced out-of-band, as we've finished a segment
+ commitLog.requestExtraSync();
}
- protected void wakeManager()
+ void awaitAvailableSegment(CommitLogSegment currentAllocatingFrom)
{
- // put a NO-OP on the queue, to trigger management thread (and create a new segment if necessary)
- segmentManagementTasks.add(Runnables.doNothing());
+ do
+ {
+ WaitQueue.Signal prepared = segmentPrepared.register(commitLog.metrics.waitingOnSegmentAllocation.time());
+ if (availableSegment == null && allocatingFrom == currentAllocatingFrom)
+ prepared.awaitUninterruptibly();
+ else
+ prepared.cancel();
+ }
+ while (availableSegment == null && allocatingFrom == currentAllocatingFrom);
}
/**
* Switch to a new segment, regardless of how much is left in the current one.
*
- * Flushes any dirty CFs for this segment and any older segments, and then recycles
- * the segments
+ * Flushes any dirty CFs for this segment and any older segments, and then discards the segments
*/
void forceRecycleAll(Iterable<UUID> droppedCfs)
{
@@ -307,7 +281,7 @@
last.waitForModifications();
// make sure the writes have materialized inside of the memtables by waiting for all outstanding writes
- // on the relevant keyspaces to complete
+ // to complete
Keyspace.writeOrder.awaitNewBarrier();
// flush and wait for all CFs that are dirty in segments up-to and including 'last'
@@ -326,7 +300,7 @@
for (CommitLogSegment segment : activeSegments)
{
if (segment.isUnused())
- recycleSegment(segment);
+ archiveAndDiscard(segment);
}
CommitLogSegment first;
@@ -341,33 +315,18 @@
}
/**
- * Indicates that a segment is no longer in use and that it should be recycled.
+ * Indicates that a segment is no longer in use and that it should be discarded.
*
* @param segment segment that is no longer in use
*/
- void recycleSegment(final CommitLogSegment segment)
+ void archiveAndDiscard(final CommitLogSegment segment)
{
boolean archiveSuccess = commitLog.archiver.maybeWaitForArchiving(segment.getName());
- if (activeSegments.remove(segment))
- {
- // if archiving (command) was not successful then leave the file alone. don't delete or recycle.
- discardSegment(segment, archiveSuccess);
- }
- else
- {
- logger.warn("segment {} not found in activeSegments queue", segment);
- }
- }
-
- /**
- * Indicates that a segment file should be deleted.
- *
- * @param segment segment to be discarded
- */
- private void discardSegment(final CommitLogSegment segment, final boolean deleteFile)
- {
- logger.trace("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
- segmentManagementTasks.add(() -> discard(segment, deleteFile));
+ if (!activeSegments.remove(segment))
+ return; // already discarded
+ // if archiving (command) was not successful then leave the file alone. don't delete or recycle.
+ logger.debug("Segment {} is no longer active and will be deleted {}", segment, archiveSuccess ? "now" : "by the archive script");
+ discard(segment, archiveSuccess);
}
/**
@@ -396,28 +355,6 @@
}
/**
- * @param name the filename to check
- * @return true if file is managed by this manager.
- */
- public boolean manages(String name)
- {
- for (CommitLogSegment segment : Iterables.concat(activeSegments, availableSegments))
- if (segment.getName().equals(name))
- return true;
- return false;
- }
-
- /**
- * Throws a flag that enables the behavior of keeping at least one spare segment
- * available at all times.
- */
- void enableReserveSegmentCreation()
- {
- createReserveSegments = true;
- wakeManager();
- }
-
- /**
* Force a flush on all CFs that are still dirty in @param segments.
*
* @return a Future that will finish when all the flushes are complete.
@@ -463,10 +400,7 @@
*/
public void stopUnsafe(boolean deleteSegments)
{
- logger.trace("CLSM closing and clearing existing commit log segments...");
- createReserveSegments = false;
-
- awaitManagementTasksCompletion();
+ logger.debug("CLSM closing and clearing existing commit log segments...");
shutdown();
try
@@ -478,35 +412,24 @@
throw new RuntimeException(e);
}
- synchronized (this)
- {
- for (CommitLogSegment segment : activeSegments)
- closeAndDeleteSegmentUnsafe(segment, deleteSegments);
- activeSegments.clear();
-
- for (CommitLogSegment segment : availableSegments)
- closeAndDeleteSegmentUnsafe(segment, deleteSegments);
- availableSegments.clear();
- }
-
- allocatingFrom = null;
-
- segmentManagementTasks.clear();
+ for (CommitLogSegment segment : activeSegments)
+ closeAndDeleteSegmentUnsafe(segment, deleteSegments);
+ activeSegments.clear();
size.set(0L);
logger.trace("CLSM done with closing and clearing existing commit log segments.");
}
- // Used by tests only.
+ /**
+ * To be used by tests only. Not safe if mutation slots are being allocated concurrently.
+ */
void awaitManagementTasksCompletion()
{
- while (!segmentManagementTasks.isEmpty())
- Thread.yield();
- // The last management task is not yet complete. Wait a while for it.
- Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
- // TODO: If this functionality is required by anything other than tests, signalling must be used to ensure
- // waiting completes correctly.
+ if (availableSegment == null && !atSegmentBufferLimit())
+ {
+ awaitAvailableSegment(allocatingFrom);
+ }
}
/**
@@ -525,18 +448,41 @@
}
/**
+ * Initiates the shutdown process for the management thread.
+ */
+ public void shutdown()
+ {
+ assert !shutdown;
+ shutdown = true;
+
+ // Release the management thread and delete prepared segment.
+ // Do not block as another thread may claim the segment (this can happen during unit test initialization).
+ discardAvailableSegment();
+ wakeManager();
+ }
+
+ private void discardAvailableSegment()
+ {
+ CommitLogSegment next = null;
+ synchronized (this) {
+ next = availableSegment;
+ availableSegment = null;
+ }
+ if (next != null)
+ next.discard(true);
+ }
+
+ /**
* Returns when the management thread terminates.
*/
public void awaitTermination() throws InterruptedException
{
managerThread.join();
+ managerThread = null;
for (CommitLogSegment segment : activeSegments)
segment.close();
- for (CommitLogSegment segment : availableSegments)
- segment.close();
-
bufferPool.shutdown();
}
@@ -554,18 +500,19 @@
*/
CommitLogPosition getCurrentPosition()
{
- return allocatingFrom().getCurrentCommitLogPosition();
+ return allocatingFrom.getCurrentCommitLogPosition();
}
/**
* Forces a disk flush on the commit log files that need it. Blocking.
*/
- public void sync(boolean syncAllSegments) throws IOException
+ public void sync() throws IOException
{
- CommitLogSegment current = allocatingFrom();
+ CommitLogSegment current = allocatingFrom;
for (CommitLogSegment segment : getActiveSegments())
{
- if (!syncAllSegments && segment.id > current.id)
+ // Do not sync segments that became active after sync started.
+ if (segment.id > current.id)
return;
segment.sync();
}
@@ -578,5 +525,25 @@
{
return bufferPool;
}
+
+ void wakeManager()
+ {
+ LockSupport.unpark(managerThread);
+ }
+
+ /**
+ * Called by commit log segments when a buffer is freed to wake the management thread, which may be waiting for
+ * a buffer to become available.
+ */
+ void notifyBufferFreed()
+ {
+ wakeManager();
+ }
+
+ /** Read-only access to current segment for subclasses. */
+ CommitLogSegment allocatingFrom()
+ {
+ return allocatingFrom;
+ }
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index 0ba4f55..7b56da3 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -17,15 +17,18 @@
*/
package org.apache.cassandra.db.commitlog;
-import org.apache.cassandra.utils.NoSpamLogger;
-import org.apache.cassandra.utils.concurrent.WaitQueue;
-import org.slf4j.*;
-
-import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.LockSupport;
-import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.codahale.metrics.Timer.Context;
+
+import org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
+import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
public abstract class AbstractCommitLogService
{
@@ -41,11 +44,10 @@
// signal that writers can wait on to be notified of a completed sync
protected final WaitQueue syncComplete = new WaitQueue();
- protected final Semaphore haveWork = new Semaphore(1);
final CommitLog commitLog;
private final String name;
- private final long pollIntervalMillis;
+ private final long pollIntervalNanos;
private static final Logger logger = LoggerFactory.getLogger(AbstractCommitLogService.class);
@@ -59,14 +61,15 @@
{
this.commitLog = commitLog;
this.name = name;
- this.pollIntervalMillis = pollIntervalMillis;
+ this.pollIntervalNanos = TimeUnit.NANOSECONDS.convert(pollIntervalMillis, TimeUnit.MILLISECONDS);
}
// Separated into individual method to ensure relevant objects are constructed before this is started.
void start()
{
- if (pollIntervalMillis < 1)
- throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %dms", pollIntervalMillis));
+ if (pollIntervalNanos < 1)
+ throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %fms",
+ pollIntervalNanos * 1e-6));
Runnable runnable = new Runnable()
{
@@ -78,26 +81,25 @@
int lagCount = 0;
int syncCount = 0;
- boolean run = true;
- while (run)
+ while (true)
{
+ // always run once after shutdown signalled
+ boolean shutdownRequested = shutdown;
+
try
{
- // always run once after shutdown signalled
- run = !shutdown;
-
// sync and signal
- long syncStarted = System.currentTimeMillis();
+ long syncStarted = System.nanoTime();
// This is a target for Byteman in CommitLogSegmentManagerTest
- commitLog.sync(shutdown);
+ commitLog.sync();
lastSyncedAt = syncStarted;
syncComplete.signalAll();
// sleep any time we have left before the next one is due
- long now = System.currentTimeMillis();
- long sleep = syncStarted + pollIntervalMillis - now;
- if (sleep < 0)
+ long now = System.nanoTime();
+ long wakeUpAt = syncStarted + pollIntervalNanos;
+ if (wakeUpAt < now)
{
// if we have lagged noticeably, update our lag counter
if (firstLagAt == 0)
@@ -105,7 +107,7 @@
firstLagAt = now;
totalSyncDuration = syncExceededIntervalBy = syncCount = lagCount = 0;
}
- syncExceededIntervalBy -= sleep;
+ syncExceededIntervalBy += now - wakeUpAt;
lagCount++;
}
syncCount++;
@@ -114,30 +116,25 @@
if (firstLagAt > 0)
{
//Only reset the lag tracking if it actually logged this time
- boolean logged = NoSpamLogger.log(
- logger,
- NoSpamLogger.Level.WARN,
- 5,
- TimeUnit.MINUTES,
- "Out of {} commit log syncs over the past {}s with average duration of {}ms, {} have exceeded the configured commit interval by an average of {}ms",
- syncCount, (now - firstLagAt) / 1000, String.format("%.2f", (double) totalSyncDuration / syncCount), lagCount, String.format("%.2f", (double) syncExceededIntervalBy / lagCount));
+ boolean logged = NoSpamLogger.log(logger,
+ NoSpamLogger.Level.WARN,
+ 5,
+ TimeUnit.MINUTES,
+ "Out of {} commit log syncs over the past {}s with average duration of {}ms, {} have exceeded the configured commit interval by an average of {}ms",
+ syncCount,
+ String.format("%.2f", (now - firstLagAt) * 1e-9d),
+ String.format("%.2f", totalSyncDuration * 1e-6d / syncCount),
+ lagCount,
+ String.format("%.2f", syncExceededIntervalBy * 1e-6d / lagCount));
if (logged)
firstLagAt = 0;
}
- // if we have lagged this round, we probably have work to do already so we don't sleep
- if (sleep < 0 || !run)
- continue;
+ if (shutdownRequested)
+ return;
- try
- {
- haveWork.tryAcquire(sleep, TimeUnit.MILLISECONDS);
- haveWork.drainPermits();
- }
- catch (InterruptedException e)
- {
- throw new AssertionError();
- }
+ if (wakeUpAt > now)
+ LockSupport.parkNanos(wakeUpAt - now);
}
catch (Throwable t)
{
@@ -145,19 +142,13 @@
break;
// sleep for full poll-interval after an error, so we don't spam the log file
- try
- {
- haveWork.tryAcquire(pollIntervalMillis, TimeUnit.MILLISECONDS);
- }
- catch (InterruptedException e)
- {
- throw new AssertionError();
- }
+ LockSupport.parkNanos(pollIntervalNanos);
}
}
}
};
+ shutdown = false;
thread = new Thread(runnable, name);
thread.start();
}
@@ -174,42 +165,43 @@
protected abstract void maybeWaitForSync(Allocation alloc);
/**
- * Sync immediately, but don't block for the sync to cmplete
+ * Request an additional sync cycle without blocking.
*/
- public WaitQueue.Signal requestExtraSync()
+ public void requestExtraSync()
{
- WaitQueue.Signal signal = syncComplete.register();
- haveWork.release(1);
- return signal;
+ LockSupport.unpark(thread);
}
public void shutdown()
{
shutdown = true;
- haveWork.release(1);
+ requestExtraSync();
}
/**
- * FOR TESTING ONLY
+ * Request sync and wait until the current state is synced.
+ *
+ * Note: If a sync is in progress at the time of this request, the call will return after both it and a cycle
+ * initiated immediately afterwards complete.
*/
- public void restartUnsafe()
+ public void syncBlocking()
{
- while (haveWork.availablePermits() < 1)
- haveWork.release();
+ long requestTime = System.nanoTime();
+ requestExtraSync();
+ awaitSyncAt(requestTime, null);
+ }
- while (haveWork.availablePermits() > 1)
+ void awaitSyncAt(long syncTime, Context context)
+ {
+ do
{
- try
- {
- haveWork.acquire();
- }
- catch (InterruptedException e)
- {
- throw new RuntimeException(e);
- }
+ WaitQueue.Signal signal = context != null ? syncComplete.register(context) : syncComplete.register();
+ if (lastSyncedAt < syncTime)
+ signal.awaitUninterruptibly();
+ else
+ signal.cancel();
}
- shutdown = false;
- start();
+ while (lastSyncedAt < syncTime);
}
public void awaitTermination() throws InterruptedException
diff --git a/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java
index ceb5d64..4edfa34 100644
--- a/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java
@@ -30,7 +30,7 @@
{
// wait until record has been safely persisted to disk
pending.incrementAndGet();
- haveWork.release();
+ requestExtraSync();
alloc.awaitDiskSync(commitLog.metrics.waitingOnCommit);
pending.decrementAndGet();
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index b66221c..0bb913d 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -38,6 +38,7 @@
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.compress.ICompressor;
import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.io.util.DataOutputBufferFixed;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.metrics.CommitLogMetrics;
@@ -116,8 +117,8 @@
CommitLog start()
{
- executor.start();
segmentManager.start();
+ executor.start();
return this;
}
@@ -129,22 +130,11 @@
*/
public int recoverSegmentsOnDisk() throws IOException
{
- // If createReserveSegments is already flipped, the CLSM is running and recovery has already taken place.
- if (segmentManager.createReserveSegments)
- return 0;
-
- FilenameFilter unmanagedFilesFilter = new FilenameFilter()
- {
- public boolean accept(File dir, String name)
- {
- // we used to try to avoid instantiating commitlog (thus creating an empty segment ready for writes)
- // until after recover was finished. this turns out to be fragile; it is less error-prone to go
- // ahead and allow writes before recover, and just skip active segments when we do.
- return CommitLogDescriptor.isValid(name) && CommitLogSegment.shouldReplay(name);
- }
- };
+ FilenameFilter unmanagedFilesFilter = (dir, name) -> CommitLogDescriptor.isValid(name) && CommitLogSegment.shouldReplay(name);
// submit all files for this segment manager for archiving prior to recovery - CASSANDRA-6904
+ // The files may have already been archived by normal CommitLog operation. This may cause errors in this
+ // archiving pass, which we should not treat as serious.
for (File file : new File(segmentManager.storageDirectory).listFiles(unmanagedFilesFilter))
{
archiver.maybeArchive(file.getPath(), file.getName());
@@ -154,6 +144,7 @@
assert archiver.archivePending.isEmpty() : "Not all commit log archive tasks were completed before restore";
archiver.maybeRestoreArchive();
+ // List the files again as archiver may have added segments.
File[] files = new File(segmentManager.storageDirectory).listFiles(unmanagedFilesFilter);
int replayed = 0;
if (files.length == 0)
@@ -171,7 +162,6 @@
segmentManager.handleReplayedSegment(f);
}
- segmentManager.enableReserveSegmentCreation();
return replayed;
}
@@ -231,9 +221,9 @@
/**
* Forces a disk flush on the commit log files that need it. Blocking.
*/
- public void sync(boolean syncAllSegments) throws IOException
+ public void sync() throws IOException
{
- segmentManager.sync(syncAllSegments);
+ segmentManager.sync();
}
/**
@@ -254,43 +244,56 @@
{
assert mutation != null;
- int size = (int) Mutation.serializer.serializedSize(mutation, MessagingService.current_version);
-
- int totalSize = size + ENTRY_OVERHEAD_SIZE;
- if (totalSize > MAX_MUTATION_SIZE)
+ DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get();
+ try
{
- throw new IllegalArgumentException(String.format("Mutation of %s is too large for the maximum size of %s",
- FBUtilities.prettyPrintMemory(totalSize),
- FBUtilities.prettyPrintMemory(MAX_MUTATION_SIZE)));
- }
+ Mutation.serializer.serialize(mutation, dob, MessagingService.current_version);
+ int size = dob.getLength();
- Allocation alloc = segmentManager.allocate(mutation, totalSize);
+ int totalSize = size + ENTRY_OVERHEAD_SIZE;
+ if (totalSize > MAX_MUTATION_SIZE)
+ {
+ throw new IllegalArgumentException(String.format("Mutation of %s is too large for the maximum size of %s",
+ FBUtilities.prettyPrintMemory(totalSize),
+ FBUtilities.prettyPrintMemory(MAX_MUTATION_SIZE)));
+ }
- CRC32 checksum = new CRC32();
- final ByteBuffer buffer = alloc.getBuffer();
- try (BufferedDataOutputStreamPlus dos = new DataOutputBufferFixed(buffer))
- {
- // checksummed length
- dos.writeInt(size);
- updateChecksumInt(checksum, size);
- buffer.putInt((int) checksum.getValue());
+ Allocation alloc = segmentManager.allocate(mutation, totalSize);
- // checksummed mutation
- Mutation.serializer.serialize(mutation, dos, MessagingService.current_version);
- updateChecksum(checksum, buffer, buffer.position() - size, size);
- buffer.putInt((int) checksum.getValue());
+ CRC32 checksum = new CRC32();
+ final ByteBuffer buffer = alloc.getBuffer();
+ try (BufferedDataOutputStreamPlus dos = new DataOutputBufferFixed(buffer))
+ {
+ // checksummed length
+ dos.writeInt(size);
+ updateChecksumInt(checksum, size);
+ buffer.putInt((int) checksum.getValue());
+
+ // checksummed mutation
+ dos.write(dob.getData(), 0, size);
+ updateChecksum(checksum, buffer, buffer.position() - size, size);
+ buffer.putInt((int) checksum.getValue());
+ }
+ catch (IOException e)
+ {
+ throw new FSWriteError(e, alloc.getSegment().getPath());
+ }
+ finally
+ {
+ alloc.markWritten();
+ }
+
+ executor.finishWriteFor(alloc);
+ return alloc.getCommitLogPosition();
}
catch (IOException e)
{
- throw new FSWriteError(e, alloc.getSegment().getPath());
+ throw new FSWriteError(e, segmentManager.allocatingFrom().getPath());
}
finally
{
- alloc.markWritten();
+ dob.recycle();
}
-
- executor.finishWriteFor(alloc);
- return alloc.getCommitLogPosition();
}
/**
@@ -315,8 +318,8 @@
if (segment.isUnused())
{
- logger.trace("Commit log segment {} is unused", segment);
- segmentManager.recycleSegment(segment);
+ logger.debug("Commit log segment {} is unused", segment);
+ segmentManager.archiveAndDiscard(segment);
}
else
{
@@ -455,23 +458,7 @@
*/
public int restartUnsafe() throws IOException
{
- segmentManager.start();
- executor.restartUnsafe();
- try
- {
- return recoverSegmentsOnDisk();
- }
- catch (FSWriteError e)
- {
- // Workaround for a class of races that keeps showing up on Windows tests.
- // stop/start/reset path on Windows with segment deletion is very touchy/brittle
- // and the timing keeps getting screwed up. Rather than chasing our tail further
- // or rewriting the CLSM, just report that we didn't recover anything back up
- // the chain. This will silence most intermittent test failures on Windows
- // and appropriately fail tests that expected segments to be recovered that
- // were not.
- return 0;
- }
+ return start().recoverSegmentsOnDisk();
}
@VisibleForTesting
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
index 044f2db..1abdd79 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
@@ -42,6 +42,7 @@
import org.slf4j.LoggerFactory;
import com.google.common.base.Strings;
+import com.google.common.base.Throwables;
public class CommitLogArchiver
{
@@ -151,22 +152,32 @@
/**
* Differs from the above because it can be used on any file, rather than only
- * managed commit log segments (and thus cannot call waitForFinalSync).
+ * managed commit log segments (and thus cannot call waitForFinalSync), and in
+ * the treatment of failures.
*
- * Used to archive files present in the commit log directory at startup (CASSANDRA-6904)
+ * Used to archive files present in the commit log directory at startup (CASSANDRA-6904).
+ * Since the files being already archived by normal operation could cause subsequent
+ * hard-linking or other operations to fail, we should not throw errors on failure
*/
public void maybeArchive(final String path, final String name)
{
if (Strings.isNullOrEmpty(archiveCommand))
return;
- archivePending.put(name, executor.submit(new WrappedRunnable()
+ archivePending.put(name, executor.submit(new Runnable()
{
- protected void runMayThrow() throws IOException
+ public void run()
{
- String command = NAME.matcher(archiveCommand).replaceAll(Matcher.quoteReplacement(name));
- command = PATH.matcher(command).replaceAll(Matcher.quoteReplacement(path));
- exec(command);
+ try
+ {
+ String command = NAME.matcher(archiveCommand).replaceAll(Matcher.quoteReplacement(name));
+ command = PATH.matcher(command).replaceAll(Matcher.quoteReplacement(path));
+ exec(command);
+ }
+ catch (IOException e)
+ {
+ logger.warn("Archiving file {} failed, file may have already been archived.", name, e);
+ }
}
}));
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
index a914cc9..e6e2e1a 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
@@ -23,7 +23,7 @@
import java.util.zip.CRC32;
import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -34,7 +34,6 @@
import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadException;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.db.rows.SerializationHelper;
-import org.apache.cassandra.io.util.ChannelProxy;
import org.apache.cassandra.io.util.DataInputBuffer;
import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.RandomAccessReader;
@@ -121,8 +120,7 @@
// just transform from the file name (no reading of headers) to determine version
CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName());
- try(ChannelProxy channel = new ChannelProxy(file);
- RandomAccessReader reader = RandomAccessReader.open(channel))
+ try(RandomAccessReader reader = RandomAccessReader.open(file))
{
if (desc.version < CommitLogDescriptor.VERSION_21)
{
@@ -191,7 +189,8 @@
ReadStatusTracker statusTracker = new ReadStatusTracker(mutationLimit, tolerateTruncation);
for (CommitLogSegmentReader.SyncSegment syncSegment : segmentReader)
{
- statusTracker.tolerateErrorsInSection &= syncSegment.toleratesErrorsInSection;
+ // Only tolerate truncation if we allow in both global and segment
+ statusTracker.tolerateErrorsInSection = tolerateTruncation & syncSegment.toleratesErrorsInSection;
// Skip segments that are completely behind the desired minPosition
if (desc.id == minPosition.segmentId && syncSegment.endPosition < minPosition.position)
@@ -470,7 +469,7 @@
}
}
- private class ReadStatusTracker
+ private static class ReadStatusTracker
{
private int mutationsLeft;
public String errorContext = "";
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index a1158be..eb9759e 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -29,8 +29,6 @@
import java.util.zip.CRC32;
import org.cliffc.high_scale_lib.NonBlockingHashMap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import com.codahale.metrics.Timer;
import org.apache.cassandra.config.*;
@@ -38,6 +36,7 @@
import org.apache.cassandra.db.commitlog.CommitLog.Configuration;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.utils.CLibrary;
import org.apache.cassandra.utils.concurrent.OpOrder;
import org.apache.cassandra.utils.concurrent.WaitQueue;
@@ -51,8 +50,6 @@
*/
public abstract class CommitLogSegment
{
- private static final Logger logger = LoggerFactory.getLogger(CommitLogSegment.class);
-
private final static long idBase;
private CDCState cdcState = CDCState.PERMITTED;
@@ -117,14 +114,13 @@
ByteBuffer buffer;
private volatile boolean headerWritten;
- final CommitLog commitLog;
public final CommitLogDescriptor descriptor;
- static CommitLogSegment createSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager, Runnable onClose)
+ static CommitLogSegment createSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
{
Configuration config = commitLog.configuration;
- CommitLogSegment segment = config.useEncryption() ? new EncryptedSegment(commitLog, manager, onClose)
- : config.useCompression() ? new CompressedSegment(commitLog, manager, onClose)
+ CommitLogSegment segment = config.useEncryption() ? new EncryptedSegment(commitLog, manager)
+ : config.useCompression() ? new CompressedSegment(commitLog, manager)
: new MemoryMappedSegment(commitLog, manager);
segment.writeLogHeader();
return segment;
@@ -152,7 +148,6 @@
*/
CommitLogSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
{
- this.commitLog = commitLog;
this.manager = manager;
id = getNextId();
@@ -370,6 +365,18 @@
}
/**
+ * Discards a segment file when the log no longer requires it. The file may be left on disk if the archive script
+ * requires it. (Potentially blocking operation)
+ */
+ void discard(boolean deleteFile)
+ {
+ close();
+ if (deleteFile)
+ FileUtils.deleteWithConfirm(logFile);
+ manager.addSize(-onDiskSize());
+ }
+
+ /**
* @return the current CommitLogPosition for this log segment
*/
public CommitLogPosition getCurrentCommitLogPosition()
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
index 04beb20..a91384f 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java
@@ -20,9 +20,11 @@
import java.io.File;
import java.io.IOException;
+import java.nio.file.FileVisitResult;
import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.BasicFileAttributes;
import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicLong;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.RateLimiter;
@@ -35,6 +37,7 @@
import org.apache.cassandra.exceptions.WriteTimeoutException;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.utils.DirectorySizeCalculator;
+import org.apache.cassandra.utils.NoSpamLogger;
public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager
{
@@ -50,8 +53,8 @@
@Override
void start()
{
- super.start();
cdcSizeTracker.start();
+ super.start();
}
public void discard(CommitLogSegment segment, boolean delete)
@@ -75,9 +78,8 @@
*/
public void shutdown()
{
- run = false;
cdcSizeTracker.shutdown();
- wakeManager();
+ super.shutdown();
}
/**
@@ -100,7 +102,7 @@
{
// Failed to allocate, so move to a new segment with enough room if possible.
advanceAllocatingFrom(segment);
- segment = allocatingFrom;
+ segment = allocatingFrom();
throwIfForbidden(mutation, segment);
}
@@ -116,6 +118,12 @@
if (mutation.trackedByCDC() && segment.getCDCState() == CDCState.FORBIDDEN)
{
cdcSizeTracker.submitOverflowSizeRecalculation();
+ NoSpamLogger.log(logger,
+ NoSpamLogger.Level.WARN,
+ 10,
+ TimeUnit.SECONDS,
+ "Rejecting Mutation containing CDC-enabled table. Free up space in {}.",
+ DatabaseDescriptor.getCDCLogLocation());
throw new WriteTimeoutException(WriteType.CDC, ConsistencyLevel.LOCAL_ONE, 0, 1);
}
}
@@ -137,7 +145,7 @@
*/
public CommitLogSegment createSegment()
{
- CommitLogSegment segment = CommitLogSegment.createSegment(commitLog, this, () -> wakeManager());
+ CommitLogSegment segment = CommitLogSegment.createSegment(commitLog, this);
cdcSizeTracker.processNewSegment(segment);
return segment;
}
@@ -148,17 +156,16 @@
*
* Allows atomic increment/decrement of unflushed size, however only allows increment on flushed and requires a full
* directory walk to determine any potential deletions by CDC consumer.
- *
- * TODO: linux performs approximately 25% better with the following one-liner instead of this walker:
- * Arrays.stream(path.listFiles()).mapToLong(File::length).sum();
- * However this solution is 375% slower on Windows. Revisit this and split logic to per-OS
*/
- private class CDCSizeTracker extends DirectorySizeCalculator
+ private static class CDCSizeTracker extends DirectorySizeCalculator
{
- private final RateLimiter rateLimiter = RateLimiter.create(1000 / DatabaseDescriptor.getCDCDiskCheckInterval());
+ private final RateLimiter rateLimiter = RateLimiter.create(1000.0 / DatabaseDescriptor.getCDCDiskCheckInterval());
private ExecutorService cdcSizeCalculationExecutor;
private CommitLogSegmentManagerCDC segmentManager;
- private AtomicLong unflushedCDCSize = new AtomicLong(0);
+ private volatile long unflushedCDCSize;
+
+ // Used instead of size during walk to remove chance of over-allocation
+ private volatile long sizeInProgress = 0;
CDCSizeTracker(CommitLogSegmentManagerCDC segmentManager, File path)
{
@@ -171,6 +178,8 @@
*/
public void start()
{
+ size = 0;
+ unflushedCDCSize = 0;
cdcSizeCalculationExecutor = new ThreadPoolExecutor(1, 1, 1000, TimeUnit.SECONDS, new SynchronousQueue<>(), new ThreadPoolExecutor.DiscardPolicy());
}
@@ -193,7 +202,7 @@
? CDCState.FORBIDDEN
: CDCState.PERMITTED);
if (segment.getCDCState() == CDCState.PERMITTED)
- unflushedCDCSize.addAndGet(defaultSegmentSize());
+ unflushedCDCSize += defaultSegmentSize();
}
// Take this opportunity to kick off a recalc to pick up any consumer file deletion.
@@ -207,9 +216,9 @@
{
// Add to flushed size before decrementing unflushed so we don't have a window of false generosity
if (segment.getCDCState() == CDCState.CONTAINS)
- size.addAndGet(segment.onDiskSize());
+ size += segment.onDiskSize();
if (segment.getCDCState() != CDCState.FORBIDDEN)
- unflushedCDCSize.addAndGet(-defaultSegmentSize());
+ unflushedCDCSize -= defaultSegmentSize();
}
// Take this opportunity to kick off a recalc to pick up any consumer file deletion.
@@ -237,7 +246,7 @@
{
rateLimiter.acquire();
calculateSize();
- CommitLogSegment allocatingFrom = segmentManager.allocatingFrom;
+ CommitLogSegment allocatingFrom = segmentManager.allocatingFrom();
if (allocatingFrom.getCDCState() == CDCState.FORBIDDEN)
processNewSegment(allocatingFrom);
}
@@ -251,14 +260,10 @@
{
try
{
- // Since we don't synchronize around either rebuilding our file list or walking the tree and adding to
- // size, it's possible we could have changes take place underneath us and end up with a slightly incorrect
- // view of our flushed size by the time this walking completes. Given that there's a linear growth in
- // runtime on both rebuildFileList and walkFileTree (about 50% for each one on runtime), and that the
- // window for this race should be very small, this is an acceptable trade-off since it will be resolved
- // on the next segment creation / deletion with a subsequent call to submitOverflowSizeRecalculation.
- rebuildFileList();
+ // The Arrays.stream approach is considerably slower on Windows than linux
+ sizeInProgress = 0;
Files.walkFileTree(path.toPath(), this);
+ size = sizeInProgress;
}
catch (IOException ie)
{
@@ -266,14 +271,21 @@
}
}
- private long addFlushedSize(long toAdd)
+ @Override
+ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException
{
- return size.addAndGet(toAdd);
+ sizeInProgress += attrs.size();
+ return FileVisitResult.CONTINUE;
+ }
+
+ private void addFlushedSize(long toAdd)
+ {
+ size += toAdd;
}
private long totalCDCSizeOnDisk()
{
- return unflushedCDCSize.get() + size.get();
+ return unflushedCDCSize + size;
}
public void shutdown()
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
index 333077c..86e886b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java
@@ -39,15 +39,6 @@
}
/**
- * Initiates the shutdown process for the management thread.
- */
- public void shutdown()
- {
- run = false;
- wakeManager();
- }
-
- /**
* Reserve space in the current segment for the provided mutation or, if there isn't space available,
* create a new segment. allocate() is blocking until allocation succeeds as it waits on a signal in advanceAllocatingFrom
*
@@ -64,7 +55,7 @@
{
// failed to allocate, so move to a new segment with enough room
advanceAllocatingFrom(segment);
- segment = allocatingFrom;
+ segment = allocatingFrom();
}
return alloc;
@@ -84,6 +75,6 @@
public CommitLogSegment createSegment()
{
- return CommitLogSegment.createSegment(commitLog, this, () -> wakeManager());
+ return CommitLogSegment.createSegment(commitLog, this);
}
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
index e44dfdf..288b766 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
@@ -20,9 +20,7 @@
import java.nio.ByteBuffer;
import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.io.compress.BufferType;
import org.apache.cassandra.io.compress.ICompressor;
-import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.utils.SyncUtil;
/**
@@ -44,15 +42,11 @@
/**
* Constructs a new segment file.
*/
- CompressedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager, Runnable onClose)
+ CompressedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
{
- super(commitLog, manager, onClose);
+ super(commitLog, manager);
this.compressor = commitLog.configuration.getCompressor();
- }
-
- ByteBuffer allocate(int size)
- {
- return compressor.preferredBufferType().allocate(size);
+ manager.getBufferPool().setPreferredReusableBufferType(compressor.preferredBufferType());
}
ByteBuffer createBuffer(CommitLog commitLog)
@@ -71,14 +65,7 @@
try
{
int neededBufferSize = compressor.initialCompressedBufferLength(length) + COMPRESSED_MARKER_SIZE;
- ByteBuffer compressedBuffer = manager.getBufferPool().getThreadLocalReusableBuffer();
- if (compressor.preferredBufferType() != BufferType.typeOf(compressedBuffer) ||
- compressedBuffer.capacity() < neededBufferSize)
- {
- FileUtils.clean(compressedBuffer);
- compressedBuffer = allocate(neededBufferSize);
- manager.getBufferPool().setThreadLocalReusableBuffer(compressedBuffer);
- }
+ ByteBuffer compressedBuffer = manager.getBufferPool().getThreadLocalReusableBuffer(neededBufferSize);
ByteBuffer inputBuffer = buffer.duplicate();
inputBuffer.limit(contentStart + length).position(contentStart);
diff --git a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
index e13b20a..4ca1ede 100644
--- a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java
@@ -25,12 +25,12 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.compress.BufferType;
import org.apache.cassandra.io.compress.ICompressor;
import org.apache.cassandra.security.EncryptionUtils;
import org.apache.cassandra.security.EncryptionContext;
-import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.Hex;
import org.apache.cassandra.utils.SyncUtil;
@@ -65,9 +65,9 @@
private final EncryptionContext encryptionContext;
private final Cipher cipher;
- public EncryptedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager, Runnable onClose)
+ public EncryptedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
{
- super(commitLog, manager, onClose);
+ super(commitLog, manager);
this.encryptionContext = commitLog.configuration.getEncryptionContext();
try
@@ -79,6 +79,8 @@
throw new FSWriteError(e, logFile);
}
logger.debug("created a new encrypted commit log segment: {}", logFile);
+ // Keep reusable buffers on-heap regardless of compression preference so we avoid copy off/on repeatedly during decryption
+ manager.getBufferPool().setPreferredReusableBufferType(BufferType.ON_HEAP);
}
protected Map<String, String> additionalHeaderParameters()
@@ -108,7 +110,7 @@
{
ByteBuffer inputBuffer = buffer.duplicate();
inputBuffer.limit(contentStart + length).position(contentStart);
- ByteBuffer buffer = manager.getBufferPool().getThreadLocalReusableBuffer();
+ ByteBuffer buffer = manager.getBufferPool().getThreadLocalReusableBuffer(DatabaseDescriptor.getCommitLogSegmentSize());
// save space for the sync marker at the beginning of this section
final long syncMarkerPosition = lastWrittenPos;
@@ -132,21 +134,17 @@
lastWrittenPos = channel.position();
- // rewind to the beginning of the section and write out the sync marker,
- // reusing the one of the existing buffers
- buffer = ByteBufferUtil.ensureCapacity(buffer, ENCRYPTED_SECTION_HEADER_SIZE, true);
+ // rewind to the beginning of the section and write out the sync marker
+ buffer.position(0).limit(ENCRYPTED_SECTION_HEADER_SIZE);
writeSyncMarker(buffer, 0, (int) syncMarkerPosition, (int) lastWrittenPos);
buffer.putInt(SYNC_MARKER_SIZE, length);
- buffer.position(0).limit(ENCRYPTED_SECTION_HEADER_SIZE);
+ buffer.rewind();
manager.addSize(buffer.limit());
channel.position(syncMarkerPosition);
channel.write(buffer);
SyncUtil.force(channel, true);
-
- if (manager.getBufferPool().getThreadLocalReusableBuffer().capacity() < buffer.capacity())
- manager.getBufferPool().setThreadLocalReusableBuffer(buffer);
}
catch (Exception e)
{
diff --git a/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java b/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java
index d4160e4..55084be 100644
--- a/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java
@@ -29,12 +29,10 @@
public abstract class FileDirectSegment extends CommitLogSegment
{
volatile long lastWrittenPos = 0;
- private final Runnable onClose;
- FileDirectSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager, Runnable onClose)
+ FileDirectSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager)
{
super(commitLog, manager);
- this.onClose = onClose;
}
@Override
@@ -62,7 +60,7 @@
}
finally
{
- onClose.run();
+ manager.notifyBufferFreed();
}
}
}
diff --git a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
index 86a248b..bde832b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
@@ -18,11 +18,10 @@
package org.apache.cassandra.db.commitlog;
import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.utils.concurrent.WaitQueue;
class PeriodicCommitLogService extends AbstractCommitLogService
{
- private static final int blockWhenSyncLagsMillis = (int) (DatabaseDescriptor.getCommitLogSyncPeriod() * 1.5);
+ private static final long blockWhenSyncLagsNanos = (long) (DatabaseDescriptor.getCommitLogSyncPeriod() * 1.5e6);
public PeriodicCommitLogService(final CommitLog commitLog)
{
@@ -31,28 +30,12 @@
protected void maybeWaitForSync(CommitLogSegment.Allocation alloc)
{
- if (waitForSyncToCatchUp(Long.MAX_VALUE))
+ long expectedSyncTime = System.nanoTime() - blockWhenSyncLagsNanos;
+ if (lastSyncedAt < expectedSyncTime)
{
- // wait until periodic sync() catches up with its schedule
- long started = System.currentTimeMillis();
pending.incrementAndGet();
- while (waitForSyncToCatchUp(started))
- {
- WaitQueue.Signal signal = syncComplete.register(commitLog.metrics.waitingOnCommit.time());
- if (waitForSyncToCatchUp(started))
- signal.awaitUninterruptibly();
- else
- signal.cancel();
- }
+ awaitSyncAt(expectedSyncTime, commitLog.metrics.waitingOnCommit.time());
pending.decrementAndGet();
}
}
-
- /**
- * @return true if sync is currently lagging behind inserts
- */
- private boolean waitForSyncToCatchUp(long started)
- {
- return started > lastSyncedAt + blockWhenSyncLagsMillis;
- }
}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/commitlog/SimpleCachedBufferPool.java b/src/java/org/apache/cassandra/db/commitlog/SimpleCachedBufferPool.java
index 1c10c25..bdec3fc 100644
--- a/src/java/org/apache/cassandra/db/commitlog/SimpleCachedBufferPool.java
+++ b/src/java/org/apache/cassandra/db/commitlog/SimpleCachedBufferPool.java
@@ -57,6 +57,8 @@
*/
private final int bufferSize;
+ private BufferType preferredReusableBufferType = BufferType.ON_HEAP;
+
public SimpleCachedBufferPool(int maxBufferPoolSize, int bufferSize)
{
this.maxBufferPoolSize = maxBufferPoolSize;
@@ -75,14 +77,19 @@
return bufferType.allocate(bufferSize);
}
- public ByteBuffer getThreadLocalReusableBuffer()
+ public ByteBuffer getThreadLocalReusableBuffer(int size)
{
- return reusableBufferHolder.get();
+ ByteBuffer result = reusableBufferHolder.get();
+ if (result.capacity() < size || BufferType.typeOf(result) != preferredReusableBufferType) {
+ FileUtils.clean(result);
+ result = preferredReusableBufferType.allocate(size);
+ reusableBufferHolder.set(result);
+ }
+ return result;
}
- public void setThreadLocalReusableBuffer(ByteBuffer buffer)
- {
- reusableBufferHolder.set(buffer);
+ public void setPreferredReusableBufferType(BufferType type) {
+ preferredReusableBufferType = type;
}
public void releaseBuffer(ByteBuffer buffer)
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index cc9fc23..4728ec3 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -27,7 +27,6 @@
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.SerializationHeader;
-import org.apache.cassandra.db.lifecycle.SSTableSet;
import org.apache.cassandra.index.Index;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.SSTableMultiWriter;
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index e6115ed..e42e7a1 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -20,7 +20,6 @@
import java.util.*;
import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.db.lifecycle.SSTableSet;
import com.google.common.collect.Iterables;
import org.apache.cassandra.db.partitions.Partition;
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java b/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
index fe81eac..535217f 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
@@ -138,8 +138,6 @@
{
private volatile boolean stopRequested = false;
public abstract CompactionInfo getCompactionInfo();
- double load = StorageMetrics.load.getCount();
- double reportedSeverity = 0d;
public void stop()
{
@@ -150,23 +148,5 @@
{
return stopRequested;
}
- /**
- * report event on the size of the compaction.
- */
- public void started()
- {
- reportedSeverity = getCompactionInfo().getTotal() / load;
- StorageService.instance.reportSeverity(reportedSeverity);
- }
-
- /**
- * remove the event complete
- */
- public void finished()
- {
- if (reportedSeverity != 0d)
- StorageService.instance.reportSeverity(-(reportedSeverity));
- reportedSeverity = 0d;
- }
}
}
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index a653b58..519ff05 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -1279,7 +1279,7 @@
{
MerkleTrees tree = new MerkleTrees(cfs.getPartitioner());
long allPartitions = 0;
- Map<Range<Token>, Long> rangePartitionCounts = new HashMap<>();
+ Map<Range<Token>, Long> rangePartitionCounts = Maps.newHashMapWithExpectedSize(ranges.size());
for (Range<Token> range : ranges)
{
long numPartitions = 0;
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java b/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java
index bb67d5f..8785b41 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java
@@ -19,7 +19,6 @@
import java.util.List;
import java.util.Map;
-import java.util.UUID;
import javax.management.openmbean.TabularData;
public interface CompactionManagerMBean
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index b6d31d5..bf367a3 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -235,11 +235,10 @@
if (!cfs.getPartitioner().splitter().isPresent())
return 0;
- List<PartitionPosition> boundaries = StorageService.getDiskBoundaries(cfs, locations.getWriteableLocations());
+ Directories.DataDirectory[] directories = locations.getWriteableLocations();
+ List<PartitionPosition> boundaries = StorageService.getDiskBoundaries(cfs, directories);
if (boundaries == null)
{
- Directories.DataDirectory[] directories = locations.getWriteableLocations();
-
// try to figure out location based on sstable directory:
for (int i = 0; i < directories.length; i++)
{
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java
index fee9e34..9362bde 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyOptions.java
@@ -27,7 +27,7 @@
public final class DateTieredCompactionStrategyOptions
{
- private static final Logger logger = LoggerFactory.getLogger(DateTieredCompactionStrategy.class);
+ private static final Logger logger = LoggerFactory.getLogger(DateTieredCompactionStrategyOptions.class);
protected static final TimeUnit DEFAULT_TIMESTAMP_RESOLUTION = TimeUnit.MICROSECONDS;
@Deprecated
protected static final double DEFAULT_MAX_SSTABLE_AGE_DAYS = 365*1000;
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index b6ad64c..ec5e1d9 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -37,7 +37,6 @@
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.ISSTableScanner;
import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.utils.FBUtilities;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.node.JsonNodeFactory;
import org.codehaus.jackson.node.ObjectNode;
@@ -180,11 +179,13 @@
for (SSTableReader sstable : ssTablesToGroup)
{
Integer level = sstable.getSSTableLevel();
- if (!sstablesByLevel.containsKey(level))
+ Collection<SSTableReader> sstablesForLevel = sstablesByLevel.get(level);
+ if (sstablesForLevel == null)
{
- sstablesByLevel.put(level, new ArrayList<SSTableReader>());
+ sstablesForLevel = new ArrayList<SSTableReader>();
+ sstablesByLevel.put(level, sstablesForLevel);
}
- sstablesByLevel.get(level).add(sstable);
+ sstablesForLevel.add(sstable);
}
Collection<Collection<SSTableReader>> groupedSSTables = new ArrayList<>();
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index d7a7d0b..fbe0513 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -25,7 +25,6 @@
import com.google.common.base.Predicates;
import com.google.common.collect.ImmutableSortedSet;
import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index 8ef2ac7..7cca4a7 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -21,7 +21,6 @@
import java.util.Map.Entry;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
index ba30dcf..02a44d7 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
@@ -94,11 +94,6 @@
// the range extend) and it's harmless to leave them.
class FilterNotIndexed extends Transformation
{
- public boolean isDoneForPartition()
- {
- return tester.isDone();
- }
-
@Override
public Row applyToRow(Row row)
{
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
index 9c4c714..9ce7ef7 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
@@ -26,7 +26,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.cql3.ColumnIdentifier;
import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.Cell;
import org.apache.cassandra.db.rows.CellPath;
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.io.util.DataInputPlus;
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
index 02f8841..447ea73 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java
@@ -123,9 +123,9 @@
Type type = Type.fromPrefix(matcher.group(1));
return new LogRecord(type,
matcher.group(2),
- Long.valueOf(matcher.group(3)),
- Integer.valueOf(matcher.group(4)),
- Long.valueOf(matcher.group(5)),
+ Long.parseLong(matcher.group(3)),
+ Integer.parseInt(matcher.group(4)),
+ Long.parseLong(matcher.group(5)),
line);
}
catch (IllegalArgumentException e)
diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java
index a5c781d..64e5190 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -33,14 +33,12 @@
import org.apache.cassandra.utils.Interval;
import static com.google.common.base.Predicates.equalTo;
-import static com.google.common.base.Predicates.in;
import static com.google.common.base.Predicates.not;
import static com.google.common.collect.ImmutableList.copyOf;
import static com.google.common.collect.ImmutableList.of;
import static com.google.common.collect.Iterables.all;
import static com.google.common.collect.Iterables.concat;
import static com.google.common.collect.Iterables.filter;
-import static com.google.common.collect.Iterables.transform;
import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
import static org.apache.cassandra.db.lifecycle.Helpers.filterOut;
import static org.apache.cassandra.db.lifecycle.Helpers.replace;
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeType.java b/src/java/org/apache/cassandra/db/marshal/TimeType.java
index 8cd221e..9992a06 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeType.java
@@ -25,7 +25,6 @@
import org.apache.cassandra.cql3.CQL3Type;
import org.apache.cassandra.serializers.TypeSerializer;
import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.ByteBufferUtil;
/**
* Nanosecond resolution time values
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index 7803ee2..74378b2 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -18,8 +18,6 @@
package org.apache.cassandra.db.marshal;
import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.nio.charset.StandardCharsets;
import java.util.*;
import java.util.stream.Collectors;
diff --git a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java b/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java
index 1d57398..cc4b410 100644
--- a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java
+++ b/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java
@@ -36,7 +36,7 @@
public class ApproximateTime
{
private static final Logger logger = LoggerFactory.getLogger(ApproximateTime.class);
- private static final int CHECK_INTERVAL_MS = Math.max(5, Integer.valueOf(System.getProperty(Config.PROPERTY_PREFIX + "approximate_time_precision_ms", "10")));
+ private static final int CHECK_INTERVAL_MS = Math.max(5, Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "approximate_time_precision_ms", "10")));
private static volatile long time = System.currentTimeMillis();
static
diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
index 6d28078..a44773a 100644
--- a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
+++ b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java
@@ -51,13 +51,13 @@
/**
* Defines the interval for reporting any operations that have timed out.
*/
- private static final int REPORT_INTERVAL_MS = Math.max(0, Integer.valueOf(System.getProperty(Config.PROPERTY_PREFIX + "monitoring_report_interval_ms", "5000")));
+ private static final int REPORT_INTERVAL_MS = Math.max(0, Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "monitoring_report_interval_ms", "5000")));
/**
* Defines the maximum number of unique timed out queries that will be reported in the logs.
* Use a negative number to remove any limit.
*/
- private static final int MAX_OPERATIONS = Integer.valueOf(System.getProperty(Config.PROPERTY_PREFIX + "monitoring_max_operations", "50"));
+ private static final int MAX_OPERATIONS = Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "monitoring_max_operations", "50"));
@VisibleForTesting
static MonitoringTask instance = make(REPORT_INTERVAL_MS, MAX_OPERATIONS);
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index 0400402..9549c0d 100644
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@ -169,7 +169,7 @@
public UnfilteredRowIterator unfilteredIterator()
{
- return unfilteredIterator(ColumnFilter.all(metadata()), Slices.ALL, false);
+ return unfilteredIterator(metadata().getAllColumnFilter(), Slices.ALL, false);
}
public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed)
@@ -197,7 +197,6 @@
ClusteringBound end = slice.end() == ClusteringBound.TOP ? null : slice.end();
Iterator<Row> rowIter = BTree.slice(current.tree, metadata.comparator, start, true, end, true, desc(reversed));
Iterator<RangeTombstone> deleteIter = current.deletionInfo.rangeIterator(slice, reversed);
-
return merge(rowIter, deleteIter, selection, reversed, current, staticRow);
}
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/PartitionIterator.java
index 529a9e2..e4b926a 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionIterator.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionIterator.java
@@ -17,8 +17,6 @@
*/
package org.apache.cassandra.db.partitions;
-import java.util.Iterator;
-
import org.apache.cassandra.db.rows.*;
/**
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
index 9b7d7eb..70759cf 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java
@@ -18,7 +18,6 @@
package org.apache.cassandra.db.partitions;
import java.util.*;
-import java.security.MessageDigest;
import org.apache.cassandra.db.EmptyIterators;
import org.apache.cassandra.db.transform.MorePartitions;
diff --git a/src/java/org/apache/cassandra/db/rows/BTreeRow.java b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
index c699634..0eccb6e 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -19,6 +19,7 @@
import java.nio.ByteBuffer;
import java.util.*;
+import java.util.function.Consumer;
import java.util.function.Predicate;
import com.google.common.base.Function;
@@ -32,9 +33,7 @@
import org.apache.cassandra.db.filter.ColumnFilter;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.utils.AbstractIterator;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.btree.BTree;
import org.apache.cassandra.utils.btree.BTreeSearchIterator;
import org.apache.cassandra.utils.btree.UpdateFunction;
@@ -168,16 +167,23 @@
return cd.column().isSimple() ? minDeletionTime((Cell) cd) : minDeletionTime((ComplexColumnData)cd);
}
+ public void apply(Consumer<ColumnData> function, boolean reversed)
+ {
+ BTree.apply(btree, function, reversed);
+ }
+
+ public void apply(Consumer<ColumnData> funtion, com.google.common.base.Predicate<ColumnData> stopCondition, boolean reversed)
+ {
+ BTree.apply(btree, funtion, stopCondition, reversed);
+ }
+
private static int minDeletionTime(Object[] btree, LivenessInfo info, DeletionTime rowDeletion)
{
- int min = Math.min(minDeletionTime(info), minDeletionTime(rowDeletion));
- for (ColumnData cd : BTree.<ColumnData>iterable(btree))
- {
- min = Math.min(min, minDeletionTime(cd));
- if (min == Integer.MIN_VALUE)
- break;
- }
- return min;
+ //we have to wrap this for the lambda
+ final WrappedInt min = new WrappedInt(Math.min(minDeletionTime(info), minDeletionTime(rowDeletion)));
+
+ BTree.<ColumnData>apply(btree, cd -> min.set( Math.min(min.get(), minDeletionTime(cd)) ), cd -> min.get() == Integer.MIN_VALUE, false);
+ return min.get();
}
public Clustering clustering()
@@ -324,17 +330,26 @@
public boolean hasComplexDeletion()
{
+ final WrappedBoolean result = new WrappedBoolean(false);
+
// We start by the end cause we know complex columns sort before simple ones
- for (ColumnData cd : BTree.<ColumnData>iterable(btree, BTree.Dir.DESC))
- {
- if (cd.column().isSimple())
- return false;
+ apply(c -> {}, cd -> {
+ if (cd.column.isSimple())
+ {
+ result.set(false);
+ return true;
+ }
if (!((ComplexColumnData) cd).complexDeletion().isLive())
+ {
+ result.set(true);
return true;
- }
+ }
- return false;
+ return false;
+ }, true);
+
+ return result.get();
}
public Row markCounterLocalToBeCleared()
diff --git a/src/java/org/apache/cassandra/db/rows/CellPath.java b/src/java/org/apache/cassandra/db/rows/CellPath.java
index e2b362c..91a5217 100644
--- a/src/java/org/apache/cassandra/db/rows/CellPath.java
+++ b/src/java/org/apache/cassandra/db/rows/CellPath.java
@@ -22,7 +22,6 @@
import java.security.MessageDigest;
import java.util.Objects;
-import org.apache.cassandra.db.marshal.UserType;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.utils.ByteBufferUtil;
diff --git a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
index b2c09b1..b5195c5 100644
--- a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
@@ -21,7 +21,6 @@
import java.security.MessageDigest;
import java.util.Iterator;
import java.util.Objects;
-import java.util.function.BiFunction;
import com.google.common.base.Function;
@@ -235,10 +234,6 @@
public static class Builder
{
- private static BiFunction<Cell, Cell, Cell> noResolve = (a, b) -> {
- throw new IllegalStateException();
- };
-
private DeletionTime complexDeletion;
private ColumnDefinition column;
private BTree.Builder<Cell> builder;
diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java
index 53b0eb3..51cf435 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -19,6 +19,9 @@
import java.util.*;
import java.security.MessageDigest;
+import java.util.function.Consumer;
+
+import com.google.common.base.Predicate;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
@@ -251,6 +254,16 @@
public String toString(CFMetaData metadata, boolean fullDetails);
/**
+ * Apply a function to every column in a row
+ */
+ public void apply(Consumer<ColumnData> function, boolean reverse);
+
+ /**
+ * Apply a funtion to every column in a row until a stop condition is reached
+ */
+ public void apply(Consumer<ColumnData> function, Predicate<ColumnData> stopCondition, boolean reverse);
+
+ /**
* A row deletion/tombstone.
* <p>
* A row deletion mostly consists of the time of said deletion, but there is 2 variants: shadowable
diff --git a/src/java/org/apache/cassandra/db/rows/Rows.java b/src/java/org/apache/cassandra/db/rows/Rows.java
index e325091..4f6c8d2 100644
--- a/src/java/org/apache/cassandra/db/rows/Rows.java
+++ b/src/java/org/apache/cassandra/db/rows/Rows.java
@@ -26,6 +26,7 @@
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
import org.apache.cassandra.utils.MergeIterator;
+import org.apache.cassandra.utils.WrappedInt;
/**
* Static utilities to work on Row objects.
@@ -72,14 +73,15 @@
collector.update(row.primaryKeyLivenessInfo());
collector.update(row.deletion().time());
- int columnCount = 0;
- int cellCount = 0;
- for (ColumnData cd : row)
- {
+ //we have to wrap these for the lambda
+ final WrappedInt columnCount = new WrappedInt(0);
+ final WrappedInt cellCount = new WrappedInt(0);
+
+ row.apply(cd -> {
if (cd.column().isSimple())
{
- ++columnCount;
- ++cellCount;
+ columnCount.increment();
+ cellCount.increment();
Cells.collectStats((Cell) cd, collector);
}
else
@@ -88,18 +90,18 @@
collector.update(complexData.complexDeletion());
if (complexData.hasCells())
{
- ++columnCount;
+ columnCount.increment();
for (Cell cell : complexData)
{
- ++cellCount;
+ cellCount.increment();
Cells.collectStats(cell, collector);
}
}
}
+ }, false);
- }
- collector.updateColumnSetPerRow(columnCount);
- return cellCount;
+ collector.updateColumnSetPerRow(columnCount.get());
+ return cellCount.get();
}
/**
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java
index a969858..bbef11e 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterator.java
@@ -17,9 +17,6 @@
*/
package org.apache.cassandra.db.rows;
-import java.util.Iterator;
-
-import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
/**
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
index 542f0a2..45c026f 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
@@ -80,12 +80,15 @@
}
// Should only be used for the on-wire format.
+
public void serialize(UnfilteredRowIterator iterator, ColumnFilter selection, DataOutputPlus out, int version, int rowEstimate) throws IOException
{
+
SerializationHeader header = new SerializationHeader(false,
iterator.metadata(),
iterator.columns(),
iterator.stats());
+
serialize(iterator, header, selection, out, version, rowEstimate);
}
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
index 5ca7e03..ed6bd12 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
@@ -28,6 +28,7 @@
import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.utils.SearchIterator;
+import org.apache.cassandra.utils.WrappedException;
/**
* Serialize/deserialize a single Unfiltered (both on-wire and on-disk).
@@ -181,7 +182,7 @@
if (header.isForSSTable())
{
- DataOutputBuffer dob = DataOutputBuffer.RECYCLER.get();
+ DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get();
try
{
serializeRowBody(row, flags, header, dob);
@@ -190,7 +191,7 @@
// We write the size of the previous unfiltered to make reverse queries more efficient (and simpler).
// This is currently not used however and using it is tbd.
out.writeUnsignedVInt(previousUnfilteredSize);
- out.write(dob.buffer());
+ out.write(dob.getData(), 0, dob.getLength());
}
finally
{
@@ -227,20 +228,37 @@
Columns.serializer.serializeSubset(Collections2.transform(row, ColumnData::column), headerColumns, out);
SearchIterator<ColumnDefinition, ColumnDefinition> si = headerColumns.iterator();
- for (ColumnData data : row)
- {
- // We can obtain the column for data directly from data.column(). However, if the cell/complex data
- // originates from a sstable, the column we'll get will have the type used when the sstable was serialized,
- // and if that type have been recently altered, that may not be the type we want to serialize the column
- // with. So we use the ColumnDefinition from the "header" which is "current". Also see #11810 for what
- // happens if we don't do that.
- ColumnDefinition column = si.next(data.column());
- assert column != null;
- if (data.column.isSimple())
- Cell.serializer.serialize((Cell) data, column, out, pkLiveness, header);
- else
- writeComplexColumn((ComplexColumnData) data, column, (flags & HAS_COMPLEX_DELETION) != 0, pkLiveness, header, out);
+ try
+ {
+ row.apply(cd -> {
+ // We can obtain the column for data directly from data.column(). However, if the cell/complex data
+ // originates from a sstable, the column we'll get will have the type used when the sstable was serialized,
+ // and if that type have been recently altered, that may not be the type we want to serialize the column
+ // with. So we use the ColumnDefinition from the "header" which is "current". Also see #11810 for what
+ // happens if we don't do that.
+ ColumnDefinition column = si.next(cd.column());
+ assert column != null : cd.column.toString();
+
+ try
+ {
+ if (cd.column.isSimple())
+ Cell.serializer.serialize((Cell) cd, column, out, pkLiveness, header);
+ else
+ writeComplexColumn((ComplexColumnData) cd, column, (flags & HAS_COMPLEX_DELETION) != 0, pkLiveness, header, out);
+ }
+ catch (IOException e)
+ {
+ throw new WrappedException(e);
+ }
+ }, false);
+ }
+ catch (WrappedException e)
+ {
+ if (e.getCause() instanceof IOException)
+ throw (IOException) e.getCause();
+
+ throw e;
}
}
@@ -496,12 +514,31 @@
builder.addRowDeletion(hasDeletion ? new Row.Deletion(header.readDeletionTime(in), deletionIsShadowable) : Row.Deletion.LIVE);
Columns columns = hasAllColumns ? headerColumns : Columns.serializer.deserializeSubset(headerColumns, in);
- for (ColumnDefinition column : columns)
+
+ final LivenessInfo livenessInfo = rowLiveness;
+
+ try
{
- if (column.isSimple())
- readSimpleColumn(column, in, header, helper, builder, rowLiveness);
- else
- readComplexColumn(column, in, header, helper, hasComplexDeletion, builder, rowLiveness);
+ columns.apply(column -> {
+ try
+ {
+ if (column.isSimple())
+ readSimpleColumn(column, in, header, helper, builder, livenessInfo);
+ else
+ readComplexColumn(column, in, header, helper, hasComplexDeletion, builder, livenessInfo);
+ }
+ catch (IOException e)
+ {
+ throw new WrappedException(e);
+ }
+ }, false);
+ }
+ catch (WrappedException e)
+ {
+ if (e.getCause() instanceof IOException)
+ throw (IOException) e.getCause();
+
+ throw e;
}
return builder.build();
diff --git a/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java b/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java
index 5e7d905..0b407f2 100644
--- a/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java
+++ b/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.db.rows;
-import org.apache.cassandra.db.*;
import org.apache.cassandra.db.filter.ColumnFilter;
import org.apache.cassandra.db.transform.Transformation;
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index 8f8abb8..4bd95d4 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -45,8 +45,6 @@
import org.apache.cassandra.repair.SystemDistributedKeyspace;
import org.apache.cassandra.service.StorageProxy;
import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.pager.QueryPager;
-import org.apache.cassandra.transport.Server;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.UUIDGen;
diff --git a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
index a8af37b..2161363 100644
--- a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@ -25,7 +25,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.ViewDefinition;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.partitions.*;
diff --git a/src/java/org/apache/cassandra/dht/Bounds.java b/src/java/org/apache/cassandra/dht/Bounds.java
index a125168..626eaa6 100644
--- a/src/java/org/apache/cassandra/dht/Bounds.java
+++ b/src/java/org/apache/cassandra/dht/Bounds.java
@@ -18,7 +18,6 @@
package org.apache.cassandra.dht;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java b/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java
index 054a90e..a60be94 100644
--- a/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java
+++ b/src/java/org/apache/cassandra/dht/tokenallocator/ReplicationAwareTokenAllocator.java
@@ -772,7 +772,7 @@
BaseTokenInfo<?, ?> token = tokens;
do
{
- System.out.format("%s%s: rs %s rt %s size %.2e\n", lead, token, token.replicationStart, token.replicationThreshold, token.replicatedOwnership);
+ System.out.format("%s%s: rs %s rt %s size %.2e%n", lead, token, token.replicationStart, token.replicationThreshold, token.replicatedOwnership);
token = token.next;
} while (token != null && token != tokens);
}
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index 964b4ad..7d8b88b 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -36,6 +36,7 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.Clock;
import org.apache.cassandra.utils.FBUtilities;
/**
@@ -52,7 +53,7 @@
private static final int DEBUG_PERCENTAGE = 80; // if the phi is larger than this percentage of the max, log a debug message
private static final long DEFAULT_MAX_PAUSE = 5000L * 1000000L; // 5 seconds
private static final long MAX_LOCAL_PAUSE_IN_NANOS = getMaxLocalPause();
- private long lastInterpret = System.nanoTime();
+ private long lastInterpret = Clock.instance.nanoTime();
private long lastPause = 0L;
private static long getMaxLocalPause()
@@ -252,7 +253,7 @@
public void report(InetAddress ep)
{
- long now = System.nanoTime();
+ long now = Clock.instance.nanoTime();
ArrivalWindow heartbeatWindow = arrivalSamples.get(ep);
if (heartbeatWindow == null)
{
@@ -279,7 +280,7 @@
{
return;
}
- long now = System.nanoTime();
+ long now = Clock.instance.nanoTime();
long diff = now - lastInterpret;
lastInterpret = now;
if (diff > MAX_LOCAL_PAUSE_IN_NANOS)
@@ -288,7 +289,7 @@
lastPause = now;
return;
}
- if (System.nanoTime() - lastPause < MAX_LOCAL_PAUSE_IN_NANOS)
+ if (Clock.instance.nanoTime() - lastPause < MAX_LOCAL_PAUSE_IN_NANOS)
{
logger.debug("Still not marking nodes down due to local pause");
return;
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
index daba701..5dba3e2 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
@@ -324,9 +324,9 @@
boolean partitionerIsOpp = partitioner instanceof OrderPreservingPartitioner || partitioner instanceof ByteOrderedPartitioner;
- for (TokenRange subSplit : subSplits.keySet())
+ for (Map.Entry<TokenRange, Long> subSplitEntry : subSplits.entrySet())
{
- List<TokenRange> ranges = subSplit.unwrap();
+ List<TokenRange> ranges = subSplitEntry.getKey().unwrap();
for (TokenRange subrange : ranges)
{
ColumnFamilySplit split =
@@ -335,7 +335,7 @@
subrange.getStart().toString().substring(2) : subrange.getStart().toString(),
partitionerIsOpp ?
subrange.getEnd().toString().substring(2) : subrange.getEnd().toString(),
- subSplits.get(subSplit),
+ subSplitEntry.getValue(),
endpoints);
logger.trace("adding {}", split);
diff --git a/src/java/org/apache/cassandra/hints/HintsBuffer.java b/src/java/org/apache/cassandra/hints/HintsBuffer.java
index e86dede..835a3be 100644
--- a/src/java/org/apache/cassandra/hints/HintsBuffer.java
+++ b/src/java/org/apache/cassandra/hints/HintsBuffer.java
@@ -28,7 +28,6 @@
import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.io.util.DataOutputBufferFixed;
-import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.utils.AbstractIterator;
diff --git a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
index 5fe0e27..47d986f 100644
--- a/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
+++ b/src/java/org/apache/cassandra/hints/HintsDispatchTrigger.java
@@ -20,7 +20,6 @@
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.cassandra.gms.ApplicationState;
-import org.apache.cassandra.gms.FailureDetector;
import org.apache.cassandra.gms.Gossiper;
import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddress;
diff --git a/src/java/org/apache/cassandra/hints/HintsWriter.java b/src/java/org/apache/cassandra/hints/HintsWriter.java
index ae9e05a..a081451 100644
--- a/src/java/org/apache/cassandra/hints/HintsWriter.java
+++ b/src/java/org/apache/cassandra/hints/HintsWriter.java
@@ -75,7 +75,7 @@
CRC32 crc = new CRC32();
DataOutputBuffer dob = null;
- try (DataOutputBuffer ignored = dob = DataOutputBuffer.RECYCLER.get())
+ try (DataOutputBuffer ignored = dob = DataOutputBuffer.scratchBuffer.get())
{
// write the descriptor
descriptor.serialize(dob);
diff --git a/src/java/org/apache/cassandra/index/internal/IndexEntry.java b/src/java/org/apache/cassandra/index/internal/IndexEntry.java
index 8ffd26a..97525d6 100644
--- a/src/java/org/apache/cassandra/index/internal/IndexEntry.java
+++ b/src/java/org/apache/cassandra/index/internal/IndexEntry.java
@@ -23,7 +23,6 @@
import java.nio.ByteBuffer;
import org.apache.cassandra.db.Clustering;
-import org.apache.cassandra.db.Clustering;
import org.apache.cassandra.db.DecoratedKey;
/**
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index e24e441..e2b0b2a 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@ -25,7 +25,6 @@
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter;
import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter;
-import org.apache.cassandra.db.filter.ColumnFilter;
import org.apache.cassandra.db.filter.DataLimits;
import org.apache.cassandra.db.filter.RowFilter;
import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/view/View.java b/src/java/org/apache/cassandra/index/sasi/conf/view/View.java
index 1f68b0c..25f32d9 100644
--- a/src/java/org/apache/cassandra/index/sasi/conf/view/View.java
+++ b/src/java/org/apache/cassandra/index/sasi/conf/view/View.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
import java.util.*;
-import org.apache.cassandra.db.marshal.UUIDType;
import org.apache.cassandra.index.sasi.SSTableIndex;
import org.apache.cassandra.index.sasi.conf.ColumnIndex;
import org.apache.cassandra.index.sasi.plan.Expression;
@@ -33,7 +32,6 @@
import org.apache.cassandra.utils.IntervalTree;
import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
public class View implements Iterable<SSTableIndex>
{
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java b/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java
index 22d6c9e..cc1eb3f 100644
--- a/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/memory/MemIndex.java
@@ -26,8 +26,6 @@
import org.apache.cassandra.index.sasi.utils.RangeIterator;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.github.jamm.MemoryMeter;
-
public abstract class MemIndex
{
protected final AbstractType<?> keyValidator;
diff --git a/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.java b/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.java
index e3d591f..5f04876 100644
--- a/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.java
+++ b/src/java/org/apache/cassandra/index/sasi/sa/IntegralSA.java
@@ -22,7 +22,6 @@
import java.util.Comparator;
import java.util.Iterator;
-import org.apache.cassandra.index.Index;
import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
import org.apache.cassandra.index.sasi.disk.TokenTreeBuilder;
import org.apache.cassandra.db.marshal.AbstractType;
diff --git a/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.java b/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.java
index 8b38530..9815233 100644
--- a/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.java
+++ b/src/java/org/apache/cassandra/index/sasi/utils/TypeUtil.java
@@ -63,7 +63,7 @@
break;
default:
- upcastToken = Long.valueOf(UTF8Type.instance.getString(term));
+ upcastToken = Long.parseLong(UTF8Type.instance.getString(term));
}
return LongType.instance.decompose(upcastToken);
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index 101f722..9a4d919 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -49,7 +49,6 @@
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.Memory;
import org.apache.cassandra.io.util.SafeMemory;
import org.apache.cassandra.schema.CompressionParams;
diff --git a/src/java/org/apache/cassandra/io/sstable/Descriptor.java b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
index 7840985..13611a6 100644
--- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java
+++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
@@ -27,11 +27,9 @@
import com.google.common.base.CharMatcher;
import com.google.common.base.Objects;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.sstable.format.Version;
-import org.apache.cassandra.io.sstable.format.big.BigFormat;
import org.apache.cassandra.io.sstable.metadata.IMetadataSerializer;
import org.apache.cassandra.io.sstable.metadata.LegacyMetadataSerializer;
import org.apache.cassandra.io.sstable.metadata.MetadataSerializer;
@@ -63,26 +61,17 @@
private final int hashCode;
/**
- * A descriptor that assumes CURRENT_VERSION.
- */
- @VisibleForTesting
- public Descriptor(File directory, String ksname, String cfname, int generation)
- {
- this(DatabaseDescriptor.getSSTableFormat().info.getLatestVersion(), directory, ksname, cfname, generation, DatabaseDescriptor.getSSTableFormat(), null);
- }
-
- /**
* Constructor for sstable writers only.
*/
public Descriptor(File directory, String ksname, String cfname, int generation, SSTableFormat.Type formatType)
{
- this(formatType.info.getLatestVersion(), directory, ksname, cfname, generation, formatType, Component.digestFor(BigFormat.latestVersion.uncompressedChecksumType()));
+ this(formatType.info.getLatestVersion(), directory, ksname, cfname, generation, formatType, Component.digestFor(formatType.info.getLatestVersion().uncompressedChecksumType()));
}
@VisibleForTesting
public Descriptor(String version, File directory, String ksname, String cfname, int generation, SSTableFormat.Type formatType)
{
- this(formatType.info.getVersion(version), directory, ksname, cfname, generation, formatType, Component.digestFor(BigFormat.latestVersion.uncompressedChecksumType()));
+ this(formatType.info.getVersion(version), directory, ksname, cfname, generation, formatType, Component.digestFor(formatType.info.getLatestVersion().uncompressedChecksumType()));
}
public Descriptor(Version version, File directory, String ksname, String cfname, int generation, SSTableFormat.Type formatType, Component digestComponent)
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 923ef82..601f5a0 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -36,6 +36,7 @@
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.util.DiskOptimizationStrategy;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.RandomAccessReader;
import org.apache.cassandra.utils.ByteBufferUtil;
@@ -68,12 +69,9 @@
public DecoratedKey first;
public DecoratedKey last;
- protected SSTable(Descriptor descriptor, CFMetaData metadata)
- {
- this(descriptor, new HashSet<>(), metadata);
- }
+ protected final DiskOptimizationStrategy optimizationStrategy;
- protected SSTable(Descriptor descriptor, Set<Component> components, CFMetaData metadata)
+ protected SSTable(Descriptor descriptor, Set<Component> components, CFMetaData metadata, DiskOptimizationStrategy optimizationStrategy)
{
// In almost all cases, metadata shouldn't be null, but allowing null allows to create a mostly functional SSTable without
// full schema definition. SSTableLoader use that ability
@@ -86,6 +84,7 @@
this.compression = dataComponents.contains(Component.COMPRESSION_INFO);
this.components = new CopyOnWriteArraySet<>(dataComponents);
this.metadata = metadata;
+ this.optimizationStrategy = Objects.requireNonNull(optimizationStrategy);
}
/**
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java
index 0bb3721..b92bc78 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java
@@ -19,7 +19,6 @@
package org.apache.cassandra.io.sstable;
import java.util.Collection;
-import java.util.List;
import java.util.UUID;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index 715a33a..b43d3d1 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -135,17 +135,14 @@
RowIndexEntry index = writer.append(partition);
if (!transaction.isOffline() && index != null)
{
- boolean save = false;
for (SSTableReader reader : transaction.originals())
{
if (reader.getCachedPosition(key, false) != null)
{
- save = true;
+ cachedKeys.put(key, index);
break;
}
}
- if (save)
- cachedKeys.put(key, index);
}
return index;
}
@@ -230,13 +227,19 @@
if (preemptiveOpenInterval == Long.MAX_VALUE)
return;
- final List<DecoratedKey> invalidateKeys = new ArrayList<>();
- invalidateKeys.addAll(cachedKeys.keySet());
newReader.setupOnline();
- for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
- newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
+ List<DecoratedKey> invalidateKeys = null;
+ if (!cachedKeys.isEmpty())
+ {
+ invalidateKeys = new ArrayList<>(cachedKeys.size());
+ for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
+ {
+ invalidateKeys.add(cacheKey.getKey());
+ newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
+ }
+ }
- cachedKeys = new HashMap<>();
+ cachedKeys.clear();
for (SSTableReader sstable : transaction.originals())
{
// we call getCurrentReplacement() to support multiple rewriters operating over the same source readers at once.
@@ -247,12 +250,15 @@
if (latest.first.compareTo(lowerbound) > 0)
continue;
- Runnable runOnClose = new InvalidateKeys(latest, invalidateKeys);
+ Runnable runOnClose = invalidateKeys != null ? new InvalidateKeys(latest, invalidateKeys) : null;
if (lowerbound.compareTo(latest.last) >= 0)
{
if (!transaction.isObsolete(latest))
{
- latest.runOnClose(runOnClose);
+ if (runOnClose != null)
+ {
+ latest.runOnClose(runOnClose);
+ }
transaction.obsolete(latest);
}
continue;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
index eb69271..2d4314e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
@@ -29,7 +29,6 @@
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.DataPosition;
import org.apache.cassandra.net.MessagingService;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 6d3a714..fa88817 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -27,6 +27,7 @@
import com.google.common.base.Throwables;
+import com.datastax.shaded.netty.util.concurrent.FastThreadLocalThread;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.rows.Row;
@@ -189,7 +190,7 @@
//// typedef
static class Buffer extends TreeMap<DecoratedKey, PartitionUpdate> {}
- private class DiskWriter extends Thread
+ private class DiskWriter extends FastThreadLocalThread
{
volatile Throwable exception = null;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 45722cd..7fbd79d 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -25,7 +25,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.dht.IPartitioner;
/**
* A SSTable writer that assumes rows are in (partitioner) sorted order.
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
index 5286ac5..09f6a55 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
@@ -22,7 +22,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index d11e057..d26edfa 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -37,11 +37,14 @@
import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
import com.clearspring.analytics.stream.cardinality.ICardinality;
+
+import org.apache.cassandra.cache.ChunkCache;
import org.apache.cassandra.cache.InstrumentingCache;
import org.apache.cassandra.cache.KeyCacheKey;
import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.*;
@@ -190,8 +193,8 @@
public final UniqueIdentifier instanceId = new UniqueIdentifier();
// indexfile and datafile: might be null before a call to load()
- protected SegmentedFile ifile;
- protected SegmentedFile dfile;
+ protected FileHandle ifile;
+ protected FileHandle dfile;
protected IndexSummary indexSummary;
protected IFilter bf;
@@ -430,16 +433,20 @@
OpenReason.NORMAL,
header == null? null : header.toHeader(metadata));
- // special implementation of load to use non-pooled SegmentedFile builders
- try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();
- SegmentedFile.Builder dbuilder = sstable.compression
- ? new CompressedSegmentedFile.Builder(null)
- : new BufferedSegmentedFile.Builder())
+ try(FileHandle.Builder ibuilder = new FileHandle.Builder(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))
+ .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
+ .withChunkCache(ChunkCache.instance);
+ FileHandle.Builder dbuilder = new FileHandle.Builder(sstable.descriptor.filenameFor(Component.DATA)).compressed(sstable.compression)
+ .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
+ .withChunkCache(ChunkCache.instance))
{
- if (!sstable.loadSummary(ibuilder, dbuilder))
+ if (!sstable.loadSummary())
sstable.buildSummary(false, false, Downsampling.BASE_SAMPLING_LEVEL);
- sstable.ifile = ibuilder.buildIndex(sstable.descriptor, sstable.indexSummary);
- sstable.dfile = dbuilder.buildData(sstable.descriptor, statsMetadata);
+ long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
+ int dataBufferSize = sstable.optimizationStrategy.bufferSize(statsMetadata.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
+ int indexBufferSize = sstable.optimizationStrategy.bufferSize(indexFileLength / sstable.indexSummary.size());
+ sstable.ifile = ibuilder.bufferSize(indexBufferSize).complete();
+ sstable.dfile = dbuilder.bufferSize(dataBufferSize).complete();
sstable.bf = FilterFactory.AlwaysPresent;
sstable.setup(false);
return sstable;
@@ -578,8 +585,8 @@
public static SSTableReader internalOpen(Descriptor desc,
Set<Component> components,
CFMetaData metadata,
- SegmentedFile ifile,
- SegmentedFile dfile,
+ FileHandle ifile,
+ FileHandle dfile,
IndexSummary isummary,
IFilter bf,
long maxDataAge,
@@ -622,7 +629,7 @@
OpenReason openReason,
SerializationHeader header)
{
- super(desc, components, metadata);
+ super(desc, components, metadata, DatabaseDescriptor.getDiskOptimizationStrategy());
this.sstableMetadata = sstableMetadata;
this.header = header;
this.maxDataAge = maxDataAge;
@@ -730,10 +737,14 @@
*/
private void load(boolean recreateBloomFilter, boolean saveSummaryIfCreated) throws IOException
{
- try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
- SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
+ try(FileHandle.Builder ibuilder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX))
+ .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
+ .withChunkCache(ChunkCache.instance);
+ FileHandle.Builder dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(compression)
+ .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap)
+ .withChunkCache(ChunkCache.instance))
{
- boolean summaryLoaded = loadSummary(ibuilder, dbuilder);
+ boolean summaryLoaded = loadSummary();
boolean builtSummary = false;
if (recreateBloomFilter || !summaryLoaded)
{
@@ -741,13 +752,19 @@
builtSummary = true;
}
- if (components.contains(Component.PRIMARY_INDEX))
- ifile = ibuilder.buildIndex(descriptor, indexSummary);
+ int dataBufferSize = optimizationStrategy.bufferSize(sstableMetadata.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
- dfile = dbuilder.buildData(descriptor, sstableMetadata);
+ if (components.contains(Component.PRIMARY_INDEX))
+ {
+ long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
+ int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / indexSummary.size());
+ ifile = ibuilder.bufferSize(indexBufferSize).complete();
+ }
+
+ dfile = dbuilder.bufferSize(dataBufferSize).complete();
if (saveSummaryIfCreated && builtSummary)
- saveSummary(ibuilder, dbuilder);
+ saveSummary();
}
catch (Throwable t)
{ // Because the tidier has not been set-up yet in SSTableReader.open(), we must release the files in case of error
@@ -835,12 +852,10 @@
* if loaded index summary has different index interval from current value stored in schema,
* then Summary.db file will be deleted and this returns false to rebuild summary.
*
- * @param ibuilder
- * @param dbuilder
* @return true if index summary is loaded successfully from Summary.db file.
*/
@SuppressWarnings("resource")
- public boolean loadSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+ public boolean loadSummary()
{
File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
if (!summariesFile.exists())
@@ -855,8 +870,6 @@
metadata.params.minIndexInterval, metadata.params.maxIndexInterval);
first = decorateKey(ByteBufferUtil.readWithLength(iStream));
last = decorateKey(ByteBufferUtil.readWithLength(iStream));
- ibuilder.deserializeBounds(iStream, descriptor.version);
- dbuilder.deserializeBounds(iStream, descriptor.version);
}
catch (IOException e)
{
@@ -879,25 +892,22 @@
/**
* Save index summary to Summary.db file.
- *
- * @param ibuilder
- * @param dbuilder
*/
- public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+ public void saveSummary()
{
- saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, indexSummary);
+ saveSummary(this.descriptor, this.first, this.last, indexSummary);
}
- private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary newSummary)
+ private void saveSummary(IndexSummary newSummary)
{
- saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, newSummary);
+ saveSummary(this.descriptor, this.first, this.last, newSummary);
}
+
/**
* Save index summary to Summary.db file.
*/
- public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last,
- SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
+ public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last, IndexSummary summary)
{
File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
if (summariesFile.exists())
@@ -908,8 +918,6 @@
IndexSummary.serializer.serialize(summary, oStream, descriptor.version.hasSamplingLevel());
ByteBufferUtil.writeWithLength(first.getKey(), oStream);
ByteBufferUtil.writeWithLength(last.getKey(), oStream);
- ibuilder.serializeBounds(oStream, descriptor.version);
- dbuilder.serializeBounds(oStream, descriptor.version);
}
catch (IOException e)
{
@@ -1044,13 +1052,13 @@
private static class DropPageCache implements Runnable
{
- final SegmentedFile dfile;
+ final FileHandle dfile;
final long dfilePosition;
- final SegmentedFile ifile;
+ final FileHandle ifile;
final long ifilePosition;
final Runnable andThen;
- private DropPageCache(SegmentedFile dfile, long dfilePosition, SegmentedFile ifile, long ifilePosition, Runnable andThen)
+ private DropPageCache(FileHandle dfile, long dfilePosition, FileHandle ifile, long ifilePosition, Runnable andThen)
{
this.dfile = dfile;
this.dfilePosition = dfilePosition;
@@ -1065,7 +1073,8 @@
if (ifile != null)
ifile.dropPageCache(ifilePosition);
- andThen.run();
+ if (andThen != null)
+ andThen.run();
}
}
@@ -1113,12 +1122,8 @@
"no adjustments to min/max_index_interval");
}
- //Always save the resampled index
- try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
- SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
- {
- saveSummary(ibuilder, dbuilder, newSummary);
- }
+ // Always save the resampled index
+ saveSummary(newSummary);
long newSize = bytesOnDisk();
StorageMetrics.load.inc(newSize - oldSize);
@@ -1240,7 +1245,7 @@
if (!compression)
throw new IllegalStateException(this + " is not compressed");
- return ((ICompressedFile) dfile).getMetadata();
+ return dfile.compressionMetadata().get();
}
/**
@@ -1595,9 +1600,7 @@
public void setCrcCheckChance(double crcCheckChance)
{
this.crcCheckChance = crcCheckChance;
- if (compression)
- ((CompressedSegmentedFile)dfile).metadata.parameters.setCrcCheckChance(crcCheckChance);
-
+ dfile.compressionMetadata().ifPresent(metadata -> metadata.parameters.setCrcCheckChance(crcCheckChance));
}
/**
@@ -1945,7 +1948,7 @@
return ifile.channel;
}
- public SegmentedFile getIndexFile()
+ public FileHandle getIndexFile()
{
return ifile;
}
@@ -2082,8 +2085,8 @@
private IFilter bf;
private IndexSummary summary;
- private SegmentedFile dfile;
- private SegmentedFile ifile;
+ private FileHandle dfile;
+ private FileHandle ifile;
private Runnable runOnClose;
private boolean isReplaced = false;
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 9f2e159..9fb5f7c 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -25,6 +25,7 @@
import com.google.common.collect.Sets;
import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.SerializationHeader;
@@ -79,7 +80,7 @@
SerializationHeader header,
Collection<SSTableFlushObserver> observers)
{
- super(descriptor, components(metadata), metadata);
+ super(descriptor, components(metadata), metadata, DatabaseDescriptor.getDiskOptimizationStrategy());
this.keyCount = keyCount;
this.repairedAt = repairedAt;
this.metadataCollector = metadataCollector;
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index c1d9bbc..5696ecb 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -23,12 +23,14 @@
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Map;
+import java.util.Optional;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.cache.ChunkCache;
import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@ -54,11 +56,12 @@
private final ColumnIndex columnIndexWriter;
private final IndexWriter iwriter;
- private final SegmentedFile.Builder dbuilder;
+ private final FileHandle.Builder dbuilder;
protected final SequentialWriter dataFile;
private DecoratedKey lastWrittenKey;
private DataPosition dataMark;
private long lastEarlyOpenLength = 0;
+ private final Optional<ChunkCache> chunkCache = Optional.ofNullable(ChunkCache.instance);
private final SequentialWriterOption writerOption = SequentialWriterOption.newBuilder()
.trickleFsync(DatabaseDescriptor.getTrickleFsync())
@@ -85,16 +88,17 @@
writerOption,
metadata.params.compression,
metadataCollector);
- dbuilder = SegmentedFile.getCompressedBuilder((CompressedSequentialWriter) dataFile);
}
else
{
dataFile = new ChecksummedSequentialWriter(new File(getFilename()),
- new File(descriptor.filenameFor(Component.CRC)),
- new File(descriptor.filenameFor(descriptor.digestComponent)),
- writerOption);
- dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), false);
+ new File(descriptor.filenameFor(Component.CRC)),
+ new File(descriptor.filenameFor(descriptor.digestComponent)),
+ writerOption);
}
+ dbuilder = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).compressed(compression)
+ .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap);
+ chunkCache.ifPresent(dbuilder::withChunkCache);
iwriter = new IndexWriter(keyCount);
columnIndexWriter = new ColumnIndex(this.header, dataFile, descriptor.version, this.observers, getRowIndexEntrySerializer().indexInfoSerializer());
@@ -277,8 +281,13 @@
assert boundary.indexLength > 0 && boundary.dataLength > 0;
// open the reader early
IndexSummary indexSummary = iwriter.summary.build(metadata.partitioner, boundary);
- SegmentedFile ifile = iwriter.builder.buildIndex(descriptor, indexSummary, boundary);
- SegmentedFile dfile = dbuilder.buildData(descriptor, stats, boundary);
+ long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
+ int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / indexSummary.size());
+ FileHandle ifile = iwriter.builder.bufferSize(indexBufferSize).complete(boundary.indexLength);
+ if (compression)
+ dbuilder.withCompressionMetadata(((CompressedSequentialWriter) dataFile).open(boundary.dataLength));
+ int dataBufferSize = optimizationStrategy.bufferSize(stats.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
+ FileHandle dfile = dbuilder.bufferSize(dataBufferSize).complete(boundary.dataLength);
invalidateCacheAtBoundary(dfile);
SSTableReader sstable = SSTableReader.internalOpen(descriptor,
components, metadata,
@@ -291,10 +300,12 @@
return sstable;
}
- void invalidateCacheAtBoundary(SegmentedFile dfile)
+ void invalidateCacheAtBoundary(FileHandle dfile)
{
- if (ChunkCache.instance != null && lastEarlyOpenLength != 0 && dfile.dataLength() > lastEarlyOpenLength)
- ChunkCache.instance.invalidatePosition(dfile, lastEarlyOpenLength);
+ chunkCache.ifPresent(cache -> {
+ if (lastEarlyOpenLength != 0 && dfile.dataLength() > lastEarlyOpenLength)
+ cache.invalidatePosition(dfile, lastEarlyOpenLength);
+ });
lastEarlyOpenLength = dfile.dataLength();
}
@@ -304,11 +315,11 @@
dataFile.sync();
iwriter.indexFile.sync();
- return openFinal(descriptor, SSTableReader.OpenReason.EARLY);
+ return openFinal(SSTableReader.OpenReason.EARLY);
}
@SuppressWarnings("resource")
- private SSTableReader openFinal(Descriptor desc, SSTableReader.OpenReason openReason)
+ private SSTableReader openFinal(SSTableReader.OpenReason openReason)
{
if (maxDataAge < 0)
maxDataAge = System.currentTimeMillis();
@@ -316,10 +327,15 @@
StatsMetadata stats = statsMetadata();
// finalize in-memory state for the reader
IndexSummary indexSummary = iwriter.summary.build(this.metadata.partitioner);
- SegmentedFile ifile = iwriter.builder.buildIndex(desc, indexSummary);
- SegmentedFile dfile = dbuilder.buildData(desc, stats);
+ long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
+ int dataBufferSize = optimizationStrategy.bufferSize(stats.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
+ int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / indexSummary.size());
+ FileHandle ifile = iwriter.builder.bufferSize(indexBufferSize).complete();
+ if (compression)
+ dbuilder.withCompressionMetadata(((CompressedSequentialWriter) dataFile).open(0));
+ FileHandle dfile = dbuilder.bufferSize(dataBufferSize).complete();
invalidateCacheAtBoundary(dfile);
- SSTableReader sstable = SSTableReader.internalOpen(desc,
+ SSTableReader sstable = SSTableReader.internalOpen(descriptor,
components,
this.metadata,
ifile,
@@ -355,7 +371,7 @@
SSTable.appendTOC(descriptor, components);
if (openResult)
- finalReader = openFinal(descriptor, SSTableReader.OpenReason.NORMAL);
+ finalReader = openFinal(SSTableReader.OpenReason.NORMAL);
}
protected Throwable doCommit(Throwable accumulate)
@@ -415,7 +431,7 @@
class IndexWriter extends AbstractTransactional implements Transactional
{
private final SequentialWriter indexFile;
- public final SegmentedFile.Builder builder;
+ public final FileHandle.Builder builder;
public final IndexSummaryBuilder summary;
public final IFilter bf;
private DataPosition mark;
@@ -423,7 +439,8 @@
IndexWriter(long keyCount)
{
indexFile = new SequentialWriter(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)), writerOption);
- builder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
+ builder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX)).mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
+ chunkCache.ifPresent(builder::withChunkCache);
summary = new IndexSummaryBuilder(keyCount, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL);
bf = FilterFactory.getFilter(keyCount, metadata.params.bloomFilterFpChance, true, descriptor.version.hasOldBfHashOrder());
// register listeners to be alerted when the data files are flushed
@@ -507,7 +524,7 @@
summary.prepareToCommit();
try (IndexSummary indexSummary = summary.build(getPartitioner()))
{
- SSTableReader.saveSummary(descriptor, first, last, builder, dbuilder, indexSummary);
+ SSTableReader.saveSummary(descriptor, first, last, indexSummary);
}
}
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 505de49..2dccf3c 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@ -21,8 +21,6 @@
import java.nio.ByteBuffer;
import java.util.*;
-import com.google.common.collect.Maps;
-
import org.apache.cassandra.db.TypeSizes;
import org.apache.cassandra.db.commitlog.CommitLogPosition;
import org.apache.cassandra.io.sstable.Component;
@@ -81,7 +79,7 @@
@Override
public Map<MetadataType, MetadataComponent> deserialize(Descriptor descriptor, EnumSet<MetadataType> types) throws IOException
{
- Map<MetadataType, MetadataComponent> components = Maps.newHashMap();
+ Map<MetadataType, MetadataComponent> components = new EnumMap<>(MetadataType.class);
File statsFile = new File(descriptor.filenameFor(Component.STATS));
if (!statsFile.exists() && types.contains(MetadataType.STATS))
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 299bc87..eb9abcf 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -20,10 +20,10 @@
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.EnumMap;
import java.util.List;
import java.util.Map;
-import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
@@ -295,7 +295,7 @@
public Map<MetadataType, MetadataComponent> finalizeMetadata(String partitioner, double bloomFilterFPChance, long repairedAt, SerializationHeader header)
{
- Map<MetadataType, MetadataComponent> components = Maps.newHashMap();
+ Map<MetadataType, MetadataComponent> components = new EnumMap<>(MetadataType.class);
components.put(MetadataType.VALIDATION, new ValidationMetadata(partitioner, bloomFilterFPChance));
components.put(MetadataType.STATS, new StatsMetadata(estimatedPartitionSize,
estimatedCellPerPartitionCount,
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index ae1787a..abc6eaa 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@ -21,7 +21,6 @@
import java.util.*;
import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -84,7 +83,7 @@
if (!statsFile.exists())
{
logger.trace("No sstable stats for {}", descriptor);
- components = Maps.newHashMap();
+ components = new EnumMap<>(MetadataType.class);
components.put(MetadataType.STATS, MetadataCollector.defaultStatsMetadata());
}
else
@@ -104,11 +103,11 @@
public Map<MetadataType, MetadataComponent> deserialize(Descriptor descriptor, FileDataInput in, EnumSet<MetadataType> types) throws IOException
{
- Map<MetadataType, MetadataComponent> components = Maps.newHashMap();
+ Map<MetadataType, MetadataComponent> components = new EnumMap<>(MetadataType.class);
// read number of components
int numComponents = in.readInt();
// read toc
- Map<MetadataType, Integer> toc = new HashMap<>(numComponents);
+ Map<MetadataType, Integer> toc = new EnumMap<>(MetadataType.class);
MetadataType[] values = MetadataType.values();
for (int i = 0; i < numComponents; i++)
{
diff --git a/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java b/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java
index 5dc0d37..7962c0f 100644
--- a/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java
+++ b/src/java/org/apache/cassandra/io/util/AbstractReaderFileProxy.java
@@ -23,7 +23,7 @@
protected final ChannelProxy channel;
protected final long fileLength;
- public AbstractReaderFileProxy(ChannelProxy channel, long fileLength)
+ protected AbstractReaderFileProxy(ChannelProxy channel, long fileLength)
{
this.channel = channel;
this.fileLength = fileLength >= 0 ? fileLength : channel.size();
diff --git a/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java b/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java
index 95af31f..1648bcf 100644
--- a/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java
+++ b/src/java/org/apache/cassandra/io/util/BufferManagingRebufferer.java
@@ -21,6 +21,7 @@
package org.apache.cassandra.io.util;
import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
import org.apache.cassandra.utils.memory.BufferPool;
@@ -36,19 +37,12 @@
protected final ByteBuffer buffer;
protected long offset = 0;
- public static BufferManagingRebufferer on(ChunkReader wrapped)
- {
- return wrapped.alignmentRequired()
- ? new Aligned(wrapped)
- : new Unaligned(wrapped);
- }
-
abstract long alignedPosition(long position);
- public BufferManagingRebufferer(ChunkReader wrapped)
+ protected BufferManagingRebufferer(ChunkReader wrapped)
{
this.source = wrapped;
- buffer = RandomAccessReader.allocateBuffer(wrapped.chunkSize(), wrapped.preferredBufferType());
+ buffer = BufferPool.get(wrapped.chunkSize(), wrapped.preferredBufferType()).order(ByteOrder.BIG_ENDIAN);
buffer.limit(0);
}
diff --git a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
deleted file mode 100644
index a46ec14..0000000
--- a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
+++ /dev/null
@@ -1,58 +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.io.util;
-
-import org.apache.cassandra.cache.ChunkCache;
-import org.apache.cassandra.io.compress.BufferType;
-
-public class BufferedSegmentedFile extends SegmentedFile
-{
- public BufferedSegmentedFile(ChannelProxy channel, int bufferSize, long length)
- {
- this(channel, createRebufferer(channel, length, bufferSize), length);
- }
-
- private BufferedSegmentedFile(ChannelProxy channel, RebuffererFactory rebufferer, long length)
- {
- super(new Cleanup(channel, rebufferer), channel, rebufferer, length);
- }
-
- private BufferedSegmentedFile(BufferedSegmentedFile copy)
- {
- super(copy);
- }
-
- private static RebuffererFactory createRebufferer(ChannelProxy channel, long length, int bufferSize)
- {
- return ChunkCache.maybeWrap(new SimpleChunkReader(channel, length, BufferType.OFF_HEAP, bufferSize));
- }
-
- public static class Builder extends SegmentedFile.Builder
- {
- public SegmentedFile complete(ChannelProxy channel, int bufferSize, long overrideLength)
- {
- long length = overrideLength > 0 ? overrideLength : channel.size();
- return new BufferedSegmentedFile(channel, bufferSize, length);
- }
- }
-
- public BufferedSegmentedFile sharedCopy()
- {
- return new BufferedSegmentedFile(this);
- }
-}
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java b/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
index 25ef615..e1f795a 100644
--- a/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
@@ -20,101 +20,26 @@
import java.io.File;
import java.io.IOException;
-import org.apache.cassandra.io.compress.BufferType;
-import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
-import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.ChecksumType;
-public class ChecksummedRandomAccessReader
+public final class ChecksummedRandomAccessReader
{
- @SuppressWarnings("serial")
- public static class CorruptFileException extends RuntimeException
+ public static RandomAccessReader open(File file, File crcFile) throws IOException
{
- public final String filePath;
-
- public CorruptFileException(Exception cause, String filePath)
+ ChannelProxy channel = new ChannelProxy(file);
+ try
{
- super(cause);
- this.filePath = filePath;
- }
- }
-
- static class ChecksummedRebufferer extends BufferManagingRebufferer
- {
- private final DataIntegrityMetadata.ChecksumValidator validator;
-
- public ChecksummedRebufferer(ChannelProxy channel, ChecksumValidator validator)
- {
- super(new SimpleChunkReader(channel, channel.size(), BufferType.ON_HEAP, validator.chunkSize));
- this.validator = validator;
- }
-
- @Override
- public BufferHolder rebuffer(long desiredPosition)
- {
- if (desiredPosition != offset + buffer.position())
- validator.seek(desiredPosition);
-
- // align with buffer size, as checksums were computed in chunks of buffer size each.
- offset = alignedPosition(desiredPosition);
- source.readChunk(offset, buffer);
-
- try
- {
- validator.validate(ByteBufferUtil.getArray(buffer), 0, buffer.remaining());
- }
- catch (IOException e)
- {
- throw new CorruptFileException(e, channel().filePath());
- }
-
- return this;
- }
-
- @Override
- public void close()
- {
- try
- {
- source.close();
- }
- finally
- {
- validator.close();
- }
- }
-
- @Override
- long alignedPosition(long desiredPosition)
- {
- return (desiredPosition / buffer.capacity()) * buffer.capacity();
- }
- }
-
- public static final class Builder extends RandomAccessReader.Builder
- {
- private final DataIntegrityMetadata.ChecksumValidator validator;
-
- @SuppressWarnings("resource")
- public Builder(File file, File crcFile) throws IOException
- {
- super(new ChannelProxy(file));
- this.validator = new DataIntegrityMetadata.ChecksumValidator(ChecksumType.CRC32,
- RandomAccessReader.open(crcFile),
- file.getPath());
- }
-
- @Override
- protected Rebufferer createRebufferer()
- {
- return new ChecksummedRebufferer(channel, validator);
- }
-
- @Override
- public RandomAccessReader build()
- {
+ DataIntegrityMetadata.ChecksumValidator validator = new DataIntegrityMetadata.ChecksumValidator(ChecksumType.CRC32,
+ RandomAccessReader.open(crcFile),
+ file.getPath());
+ Rebufferer rebufferer = new ChecksummedRebufferer(channel, validator);
// Always own and close the channel.
- return buildWithChannel();
+ return new RandomAccessReader.RandomAccessReaderWithOwnChannel(rebufferer);
+ }
+ catch (Throwable t)
+ {
+ channel.close();
+ throw t;
}
}
}
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedRebufferer.java b/src/java/org/apache/cassandra/io/util/ChecksummedRebufferer.java
new file mode 100644
index 0000000..bc8695f
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedRebufferer.java
@@ -0,0 +1,76 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+class ChecksummedRebufferer extends BufferManagingRebufferer
+{
+ private final DataIntegrityMetadata.ChecksumValidator validator;
+
+ ChecksummedRebufferer(ChannelProxy channel, DataIntegrityMetadata.ChecksumValidator validator)
+ {
+ super(new SimpleChunkReader(channel, channel.size(), BufferType.ON_HEAP, validator.chunkSize));
+ this.validator = validator;
+ }
+
+ @Override
+ public BufferHolder rebuffer(long desiredPosition)
+ {
+ if (desiredPosition != offset + buffer.position())
+ validator.seek(desiredPosition);
+
+ // align with buffer size, as checksums were computed in chunks of buffer size each.
+ offset = alignedPosition(desiredPosition);
+ source.readChunk(offset, buffer);
+
+ try
+ {
+ validator.validate(ByteBufferUtil.getArray(buffer), 0, buffer.remaining());
+ }
+ catch (IOException e)
+ {
+ throw new CorruptFileException(e, channel().filePath());
+ }
+
+ return this;
+ }
+
+ @Override
+ public void close()
+ {
+ try
+ {
+ source.close();
+ }
+ finally
+ {
+ validator.close();
+ }
+ }
+
+ @Override
+ long alignedPosition(long desiredPosition)
+ {
+ return (desiredPosition / buffer.capacity()) * buffer.capacity();
+ }
+}
diff --git a/src/java/org/apache/cassandra/io/util/ChunkReader.java b/src/java/org/apache/cassandra/io/util/ChunkReader.java
index a04299a..1d3439e 100644
--- a/src/java/org/apache/cassandra/io/util/ChunkReader.java
+++ b/src/java/org/apache/cassandra/io/util/ChunkReader.java
@@ -44,11 +44,6 @@
int chunkSize();
/**
- * If true, positions passed to this rebufferer must be aligned to chunkSize.
- */
- boolean alignmentRequired();
-
- /**
* Specifies type of buffer the caller should attempt to give.
* This is not guaranteed to be fulfilled.
*/
diff --git a/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java b/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java
new file mode 100644
index 0000000..5f8751a
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/CompressedChunkReader.java
@@ -0,0 +1,227 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.primitives.Ints;
+
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.compress.CorruptBlockException;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+
+public abstract class CompressedChunkReader extends AbstractReaderFileProxy implements ChunkReader
+{
+ final CompressionMetadata metadata;
+
+ protected CompressedChunkReader(ChannelProxy channel, CompressionMetadata metadata)
+ {
+ super(channel, metadata.dataLength);
+ this.metadata = metadata;
+ assert Integer.bitCount(metadata.chunkLength()) == 1; //must be a power of two
+ }
+
+ @VisibleForTesting
+ public double getCrcCheckChance()
+ {
+ return metadata.parameters.getCrcCheckChance();
+ }
+
+ @Override
+ public String toString()
+ {
+ return String.format("CompressedChunkReader.%s(%s - %s, chunk length %d, data length %d)",
+ getClass().getSimpleName(),
+ channel.filePath(),
+ metadata.compressor().getClass().getSimpleName(),
+ metadata.chunkLength(),
+ metadata.dataLength);
+ }
+
+ @Override
+ public int chunkSize()
+ {
+ return metadata.chunkLength();
+ }
+
+ @Override
+ public BufferType preferredBufferType()
+ {
+ return metadata.compressor().preferredBufferType();
+ }
+
+ @Override
+ public Rebufferer instantiateRebufferer()
+ {
+ return new BufferManagingRebufferer.Aligned(this);
+ }
+
+ public static class Standard extends CompressedChunkReader
+ {
+ // we read the raw compressed bytes into this buffer, then uncompressed them into the provided one.
+ private final ThreadLocal<ByteBuffer> compressedHolder;
+
+ public Standard(ChannelProxy channel, CompressionMetadata metadata)
+ {
+ super(channel, metadata);
+ compressedHolder = ThreadLocal.withInitial(this::allocateBuffer);
+ }
+
+ public ByteBuffer allocateBuffer()
+ {
+ return allocateBuffer(metadata.compressor().initialCompressedBufferLength(metadata.chunkLength()));
+ }
+
+ public ByteBuffer allocateBuffer(int size)
+ {
+ return metadata.compressor().preferredBufferType().allocate(size);
+ }
+
+ @Override
+ public void readChunk(long position, ByteBuffer uncompressed)
+ {
+ try
+ {
+ // accesses must always be aligned
+ assert (position & -uncompressed.capacity()) == position;
+ assert position <= fileLength;
+
+ CompressionMetadata.Chunk chunk = metadata.chunkFor(position);
+ ByteBuffer compressed = compressedHolder.get();
+
+ if (compressed.capacity() < chunk.length)
+ {
+ compressed = allocateBuffer(chunk.length);
+ compressedHolder.set(compressed);
+ }
+ else
+ {
+ compressed.clear();
+ }
+
+ compressed.limit(chunk.length);
+ if (channel.read(compressed, chunk.offset) != chunk.length)
+ throw new CorruptBlockException(channel.filePath(), chunk);
+
+ compressed.flip();
+ uncompressed.clear();
+
+ try
+ {
+ metadata.compressor().uncompress(compressed, uncompressed);
+ }
+ catch (IOException e)
+ {
+ throw new CorruptBlockException(channel.filePath(), chunk);
+ }
+ finally
+ {
+ uncompressed.flip();
+ }
+
+ if (getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
+ {
+ compressed.rewind();
+ int checksum = (int) metadata.checksumType.of(compressed);
+
+ compressed.clear().limit(Integer.BYTES);
+ if (channel.read(compressed, chunk.offset + chunk.length) != Integer.BYTES
+ || compressed.getInt(0) != checksum)
+ throw new CorruptBlockException(channel.filePath(), chunk);
+ }
+ }
+ catch (CorruptBlockException e)
+ {
+ throw new CorruptSSTableException(e, channel.filePath());
+ }
+ }
+ }
+
+ public static class Mmap extends CompressedChunkReader
+ {
+ protected final MmappedRegions regions;
+
+ public Mmap(ChannelProxy channel, CompressionMetadata metadata, MmappedRegions regions)
+ {
+ super(channel, metadata);
+ this.regions = regions;
+ }
+
+ @Override
+ public void readChunk(long position, ByteBuffer uncompressed)
+ {
+ try
+ {
+ // accesses must always be aligned
+ assert (position & -uncompressed.capacity()) == position;
+ assert position <= fileLength;
+
+ CompressionMetadata.Chunk chunk = metadata.chunkFor(position);
+
+ MmappedRegions.Region region = regions.floor(chunk.offset);
+ long segmentOffset = region.offset();
+ int chunkOffset = Ints.checkedCast(chunk.offset - segmentOffset);
+ ByteBuffer compressedChunk = region.buffer();
+
+ compressedChunk.position(chunkOffset).limit(chunkOffset + chunk.length);
+
+ uncompressed.clear();
+
+ try
+ {
+ metadata.compressor().uncompress(compressedChunk, uncompressed);
+ }
+ catch (IOException e)
+ {
+ throw new CorruptBlockException(channel.filePath(), chunk);
+ }
+ finally
+ {
+ uncompressed.flip();
+ }
+
+ if (getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
+ {
+ compressedChunk.position(chunkOffset).limit(chunkOffset + chunk.length);
+
+ int checksum = (int) metadata.checksumType.of(compressedChunk);
+
+ compressedChunk.limit(compressedChunk.capacity());
+ if (compressedChunk.getInt() != checksum)
+ throw new CorruptBlockException(channel.filePath(), chunk);
+ }
+ }
+ catch (CorruptBlockException e)
+ {
+ throw new CorruptSSTableException(e, channel.filePath());
+ }
+
+ }
+
+ public void close()
+ {
+ regions.closeQuietly();
+ super.close();
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
deleted file mode 100644
index 7365d40..0000000
--- a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
+++ /dev/null
@@ -1,358 +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.io.util;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.concurrent.ThreadLocalRandom;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.primitives.Ints;
-
-import org.apache.cassandra.cache.ChunkCache;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.Config.DiskAccessMode;
-import org.apache.cassandra.io.compress.*;
-import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.utils.concurrent.Ref;
-
-public class CompressedSegmentedFile extends SegmentedFile implements ICompressedFile
-{
- public final CompressionMetadata metadata;
-
- public CompressedSegmentedFile(ChannelProxy channel, CompressionMetadata metadata, Config.DiskAccessMode mode)
- {
- this(channel,
- metadata,
- mode == DiskAccessMode.mmap
- ? MmappedRegions.map(channel, metadata)
- : null);
- }
-
- public CompressedSegmentedFile(ChannelProxy channel, CompressionMetadata metadata, MmappedRegions regions)
- {
- this(channel, metadata, regions, createRebufferer(channel, metadata, regions));
- }
-
- private static RebuffererFactory createRebufferer(ChannelProxy channel, CompressionMetadata metadata, MmappedRegions regions)
- {
- return ChunkCache.maybeWrap(chunkReader(channel, metadata, regions));
- }
-
- public static ChunkReader chunkReader(ChannelProxy channel, CompressionMetadata metadata, MmappedRegions regions)
- {
- return regions != null
- ? new Mmap(channel, metadata, regions)
- : new Standard(channel, metadata);
- }
-
- public CompressedSegmentedFile(ChannelProxy channel, CompressionMetadata metadata, MmappedRegions regions, RebuffererFactory rebufferer)
- {
- super(new Cleanup(channel, metadata, regions, rebufferer), channel, rebufferer, metadata.compressedFileLength);
- this.metadata = metadata;
- }
-
- private CompressedSegmentedFile(CompressedSegmentedFile copy)
- {
- super(copy);
- this.metadata = copy.metadata;
- }
-
- public ChannelProxy channel()
- {
- return channel;
- }
-
- private static final class Cleanup extends SegmentedFile.Cleanup
- {
- final CompressionMetadata metadata;
-
- protected Cleanup(ChannelProxy channel, CompressionMetadata metadata, MmappedRegions regions, ReaderFileProxy rebufferer)
- {
- super(channel, rebufferer);
- this.metadata = metadata;
- }
- public void tidy()
- {
- if (ChunkCache.instance != null)
- {
- ChunkCache.instance.invalidateFile(name());
- }
- metadata.close();
-
- super.tidy();
- }
- }
-
- public CompressedSegmentedFile sharedCopy()
- {
- return new CompressedSegmentedFile(this);
- }
-
- public void addTo(Ref.IdentityCollection identities)
- {
- super.addTo(identities);
- metadata.addTo(identities);
- }
-
- public static class Builder extends SegmentedFile.Builder
- {
- final CompressedSequentialWriter writer;
- final Config.DiskAccessMode mode;
-
- public Builder(CompressedSequentialWriter writer)
- {
- this.writer = writer;
- this.mode = DatabaseDescriptor.getDiskAccessMode();
- }
-
- protected CompressionMetadata metadata(String path, long overrideLength)
- {
- if (writer == null)
- return CompressionMetadata.create(path);
-
- return writer.open(overrideLength);
- }
-
- public SegmentedFile complete(ChannelProxy channel, int bufferSize, long overrideLength)
- {
- return new CompressedSegmentedFile(channel, metadata(channel.filePath(), overrideLength), mode);
- }
- }
-
- public void dropPageCache(long before)
- {
- if (before >= metadata.dataLength)
- super.dropPageCache(0);
- super.dropPageCache(metadata.chunkFor(before).offset);
- }
-
- public CompressionMetadata getMetadata()
- {
- return metadata;
- }
-
- public long dataLength()
- {
- return metadata.dataLength;
- }
-
- @VisibleForTesting
- public abstract static class CompressedChunkReader extends AbstractReaderFileProxy implements ChunkReader
- {
- final CompressionMetadata metadata;
-
- public CompressedChunkReader(ChannelProxy channel, CompressionMetadata metadata)
- {
- super(channel, metadata.dataLength);
- this.metadata = metadata;
- assert Integer.bitCount(metadata.chunkLength()) == 1; //must be a power of two
- }
-
- @VisibleForTesting
- public double getCrcCheckChance()
- {
- return metadata.parameters.getCrcCheckChance();
- }
-
- @Override
- public String toString()
- {
- return String.format("CompressedChunkReader.%s(%s - %s, chunk length %d, data length %d)",
- getClass().getSimpleName(),
- channel.filePath(),
- metadata.compressor().getClass().getSimpleName(),
- metadata.chunkLength(),
- metadata.dataLength);
- }
-
- @Override
- public int chunkSize()
- {
- return metadata.chunkLength();
- }
-
- @Override
- public boolean alignmentRequired()
- {
- return true;
- }
-
- @Override
- public BufferType preferredBufferType()
- {
- return metadata.compressor().preferredBufferType();
- }
-
- @Override
- public Rebufferer instantiateRebufferer()
- {
- return BufferManagingRebufferer.on(this);
- }
- }
-
- static class Standard extends CompressedChunkReader
- {
- // we read the raw compressed bytes into this buffer, then uncompressed them into the provided one.
- private final ThreadLocal<ByteBuffer> compressedHolder;
-
- public Standard(ChannelProxy channel, CompressionMetadata metadata)
- {
- super(channel, metadata);
- compressedHolder = ThreadLocal.withInitial(this::allocateBuffer);
- }
-
- public ByteBuffer allocateBuffer()
- {
- return allocateBuffer(metadata.compressor().initialCompressedBufferLength(metadata.chunkLength()));
- }
-
- public ByteBuffer allocateBuffer(int size)
- {
- return metadata.compressor().preferredBufferType().allocate(size);
- }
-
- @Override
- public void readChunk(long position, ByteBuffer uncompressed)
- {
- try
- {
- // accesses must always be aligned
- assert (position & -uncompressed.capacity()) == position;
- assert position <= fileLength;
-
- CompressionMetadata.Chunk chunk = metadata.chunkFor(position);
- ByteBuffer compressed = compressedHolder.get();
-
- if (compressed.capacity() < chunk.length)
- {
- compressed = allocateBuffer(chunk.length);
- compressedHolder.set(compressed);
- }
- else
- {
- compressed.clear();
- }
-
- compressed.limit(chunk.length);
- if (channel.read(compressed, chunk.offset) != chunk.length)
- throw new CorruptBlockException(channel.filePath(), chunk);
-
- compressed.flip();
- uncompressed.clear();
-
- try
- {
- metadata.compressor().uncompress(compressed, uncompressed);
- }
- catch (IOException e)
- {
- throw new CorruptBlockException(channel.filePath(), chunk);
- }
- finally
- {
- uncompressed.flip();
- }
-
- if (getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
- {
- compressed.rewind();
- int checksum = (int) metadata.checksumType.of(compressed);
-
- compressed.clear().limit(Integer.BYTES);
- if (channel.read(compressed, chunk.offset + chunk.length) != Integer.BYTES
- || compressed.getInt(0) != checksum)
- throw new CorruptBlockException(channel.filePath(), chunk);
- }
- }
- catch (CorruptBlockException e)
- {
- throw new CorruptSSTableException(e, channel.filePath());
- }
- }
- }
-
- static class Mmap extends CompressedChunkReader
- {
- protected final MmappedRegions regions;
-
- public Mmap(ChannelProxy channel, CompressionMetadata metadata, MmappedRegions regions)
- {
- super(channel, metadata);
- this.regions = regions;
- }
-
- @Override
- public void readChunk(long position, ByteBuffer uncompressed)
- {
- try
- {
- // accesses must always be aligned
- assert (position & -uncompressed.capacity()) == position;
- assert position <= fileLength;
-
- CompressionMetadata.Chunk chunk = metadata.chunkFor(position);
-
- MmappedRegions.Region region = regions.floor(chunk.offset);
- long segmentOffset = region.offset();
- int chunkOffset = Ints.checkedCast(chunk.offset - segmentOffset);
- ByteBuffer compressedChunk = region.buffer();
-
- compressedChunk.position(chunkOffset).limit(chunkOffset + chunk.length);
-
- uncompressed.clear();
-
- try
- {
- metadata.compressor().uncompress(compressedChunk, uncompressed);
- }
- catch (IOException e)
- {
- throw new CorruptBlockException(channel.filePath(), chunk);
- }
- finally
- {
- uncompressed.flip();
- }
-
- if (getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
- {
- compressedChunk.position(chunkOffset).limit(chunkOffset + chunk.length);
-
- int checksum = (int) metadata.checksumType.of(compressedChunk);
-
- compressedChunk.limit(compressedChunk.capacity());
- if (compressedChunk.getInt() != checksum)
- throw new CorruptBlockException(channel.filePath(), chunk);
- }
- }
- catch (CorruptBlockException e)
- {
- throw new CorruptSSTableException(e, channel.filePath());
- }
-
- }
-
- public void close()
- {
- regions.closeQuietly();
- super.close();
- }
- }
-}
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/src/java/org/apache/cassandra/io/util/CorruptFileException.java
similarity index 76%
copy from src/java/org/apache/cassandra/io/util/ICompressedFile.java
copy to src/java/org/apache/cassandra/io/util/CorruptFileException.java
index e69487c..875d06f 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/src/java/org/apache/cassandra/io/util/CorruptFileException.java
@@ -15,12 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.compress.CompressionMetadata;
-
-public interface ICompressedFile
+@SuppressWarnings("serial")
+public class CorruptFileException extends RuntimeException
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ public final String filePath;
+
+ public CorruptFileException(Exception cause, String filePath)
+ {
+ super(cause);
+ this.filePath = filePath;
+ }
}
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
index f08b48f..0d2423c 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
@@ -24,7 +24,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
-import io.netty.util.Recycler;
+import io.netty.util.concurrent.FastThreadLocal;
import org.apache.cassandra.config.Config;
/**
@@ -43,25 +43,22 @@
/*
* Only recycle OutputBuffers up to 1Mb. Larger buffers will be trimmed back to this size.
*/
- private static final int MAX_RECYCLE_BUFFER_SIZE = 1024 * 1024;
+ private static final int MAX_RECYCLE_BUFFER_SIZE = Integer.getInteger(Config.PROPERTY_PREFIX + "dob_max_recycle_bytes", 1024 * 1024);
private static final int DEFAULT_INITIAL_BUFFER_SIZE = 128;
- public static final Recycler<DataOutputBuffer> RECYCLER = new Recycler<DataOutputBuffer>()
+ /**
+ * Scratch buffers used mostly for serializing in memory. It's important to call #recycle() when finished
+ * to keep the memory overhead from being too large in the system.
+ */
+ public static final FastThreadLocal<DataOutputBuffer> scratchBuffer = new FastThreadLocal<DataOutputBuffer>()
{
- protected DataOutputBuffer newObject(Handle handle)
+ protected DataOutputBuffer initialValue() throws Exception
{
- return new DataOutputBuffer(handle);
+ return new DataOutputBuffer();
}
};
- private final Recycler.Handle handle;
-
- private DataOutputBuffer(Recycler.Handle handle)
- {
- this(DEFAULT_INITIAL_BUFFER_SIZE, handle);
- }
-
public DataOutputBuffer()
{
this(DEFAULT_INITIAL_BUFFER_SIZE);
@@ -69,28 +66,23 @@
public DataOutputBuffer(int size)
{
- this(size, null);
+ super(ByteBuffer.allocate(size));
}
- protected DataOutputBuffer(int size, Recycler.Handle handle)
- {
- this(ByteBuffer.allocate(size), handle);
- }
-
- protected DataOutputBuffer(ByteBuffer buffer, Recycler.Handle handle)
+ public DataOutputBuffer(ByteBuffer buffer)
{
super(buffer);
- this.handle = handle;
}
public void recycle()
{
- assert handle != null;
-
- if (buffer().capacity() <= MAX_RECYCLE_BUFFER_SIZE)
+ if (buffer.capacity() <= MAX_RECYCLE_BUFFER_SIZE)
{
- buffer.rewind();
- RECYCLER.recycle(this, handle);
+ buffer.clear();
+ }
+ else
+ {
+ buffer = ByteBuffer.allocate(DEFAULT_INITIAL_BUFFER_SIZE);
}
}
@@ -175,6 +167,11 @@
return new GrowingChannel();
}
+ public void clear()
+ {
+ buffer.clear();
+ }
+
@VisibleForTesting
final class GrowingChannel implements WritableByteChannel
{
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java b/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
index 5193401..8beb7a9 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
@@ -38,12 +38,12 @@
public DataOutputBufferFixed(int size)
{
- super(size, null);
+ super(size);
}
public DataOutputBufferFixed(ByteBuffer buffer)
{
- super(buffer, null);
+ super(buffer);
}
@Override
diff --git a/src/java/org/apache/cassandra/io/util/DiskOptimizationStrategy.java b/src/java/org/apache/cassandra/io/util/DiskOptimizationStrategy.java
new file mode 100644
index 0000000..e10342d
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/DiskOptimizationStrategy.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.util;
+
+public interface DiskOptimizationStrategy
+{
+ // The maximum buffer size, we will never buffer more than this size. Further,
+ // when the limiter is not null, i.e. when throttling is enabled, we read exactly
+ // this size, since when throttling the intention is to eventually read everything,
+ // see CASSANDRA-8630
+ // NOTE: this size is chosen both for historical consistency, as a reasonable upper bound,
+ // and because our BufferPool currently has a maximum allocation size of this.
+ int MAX_BUFFER_SIZE = 1 << 16; // 64k
+
+ /**
+ * @param recordSize record size
+ * @return the buffer size for a given record size.
+ */
+ int bufferSize(long recordSize);
+
+ /**
+ * Round up to the next multiple of 4k but no more than {@link #MAX_BUFFER_SIZE}.
+ */
+ default int roundBufferSize(long size)
+ {
+ if (size <= 0)
+ return 4096;
+
+ size = (size + 4095) & ~4095;
+ return (int)Math.min(size, MAX_BUFFER_SIZE);
+ }
+}
diff --git a/src/java/org/apache/cassandra/io/util/FileHandle.java b/src/java/org/apache/cassandra/io/util/FileHandle.java
new file mode 100644
index 0000000..62fe5e5
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/FileHandle.java
@@ -0,0 +1,440 @@
+/*
+ * 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.io.util;
+
+import java.util.Objects;
+import java.util.Optional;
+
+import com.google.common.util.concurrent.RateLimiter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.RefCounted;
+import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
+
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
+/**
+ * {@link FileHandle} provides access to a file for reading, including the ones written by various {@link SequentialWriter}
+ * instances, and it is typically used by {@link org.apache.cassandra.io.sstable.format.SSTableReader}.
+ *
+ * Use {@link FileHandle.Builder} to create an instance, and call {@link #createReader()} (and its variants) to
+ * access the readers for the underlying file.
+ *
+ * You can use {@link Builder#complete()} several times during its lifecycle with different {@code overrideLength}(i.e. early opening file).
+ * For that reason, the builder keeps a reference to the file channel and makes a copy for each {@link Builder#complete()} call.
+ * Therefore, it is important to close the {@link Builder} when it is no longer needed, as well as any {@link FileHandle}
+ * instances.
+ */
+public class FileHandle extends SharedCloseableImpl
+{
+ private static final Logger logger = LoggerFactory.getLogger(FileHandle.class);
+
+ public final ChannelProxy channel;
+
+ public final long onDiskLength;
+
+ /*
+ * Rebufferer factory to use when constructing RandomAccessReaders
+ */
+ private final RebuffererFactory rebuffererFactory;
+
+ /*
+ * Optional CompressionMetadata when dealing with compressed file
+ */
+ private final Optional<CompressionMetadata> compressionMetadata;
+
+ private FileHandle(Cleanup cleanup,
+ ChannelProxy channel,
+ RebuffererFactory rebuffererFactory,
+ CompressionMetadata compressionMetadata,
+ long onDiskLength)
+ {
+ super(cleanup);
+ this.rebuffererFactory = rebuffererFactory;
+ this.channel = channel;
+ this.compressionMetadata = Optional.ofNullable(compressionMetadata);
+ this.onDiskLength = onDiskLength;
+ }
+
+ private FileHandle(FileHandle copy)
+ {
+ super(copy);
+ channel = copy.channel;
+ rebuffererFactory = copy.rebuffererFactory;
+ compressionMetadata = copy.compressionMetadata;
+ onDiskLength = copy.onDiskLength;
+ }
+
+ /**
+ * @return Path to the file this factory is referencing
+ */
+ public String path()
+ {
+ return channel.filePath();
+ }
+
+ public long dataLength()
+ {
+ return compressionMetadata.map(c -> c.dataLength).orElseGet(rebuffererFactory::fileLength);
+ }
+
+ public RebuffererFactory rebuffererFactory()
+ {
+ return rebuffererFactory;
+ }
+
+ public Optional<CompressionMetadata> compressionMetadata()
+ {
+ return compressionMetadata;
+ }
+
+ @Override
+ public void addTo(Ref.IdentityCollection identities)
+ {
+ super.addTo(identities);
+ compressionMetadata.ifPresent(metadata -> metadata.addTo(identities));
+ }
+
+ @Override
+ public FileHandle sharedCopy()
+ {
+ return new FileHandle(this);
+ }
+
+ /**
+ * Create {@link RandomAccessReader} with configured method of reading content of the file.
+ *
+ * @return RandomAccessReader for the file
+ */
+ public RandomAccessReader createReader()
+ {
+ return createReader(null);
+ }
+
+ /**
+ * Create {@link RandomAccessReader} with configured method of reading content of the file.
+ * Reading from file will be rate limited by given {@link RateLimiter}.
+ *
+ * @param limiter RateLimiter to use for rate limiting read
+ * @return RandomAccessReader for the file
+ */
+ public RandomAccessReader createReader(RateLimiter limiter)
+ {
+ return new RandomAccessReader(instantiateRebufferer(limiter));
+ }
+
+ public FileDataInput createReader(long position)
+ {
+ RandomAccessReader reader = createReader();
+ reader.seek(position);
+ return reader;
+ }
+
+ /**
+ * Drop page cache from start to given {@code before}.
+ *
+ * @param before uncompressed position from start of the file to be dropped from cache. if 0, to end of file.
+ */
+ public void dropPageCache(long before)
+ {
+ long position = compressionMetadata.map(metadata -> {
+ if (before >= metadata.dataLength)
+ return 0L;
+ else
+ return metadata.chunkFor(before).offset;
+ }).orElse(before);
+ CLibrary.trySkipCache(channel.getFileDescriptor(), 0, position, path());
+ }
+
+ private Rebufferer instantiateRebufferer(RateLimiter limiter)
+ {
+ Rebufferer rebufferer = rebuffererFactory.instantiateRebufferer();
+
+ if (limiter != null)
+ rebufferer = new LimitingRebufferer(rebufferer, limiter, DiskOptimizationStrategy.MAX_BUFFER_SIZE);
+ return rebufferer;
+ }
+
+ /**
+ * Perform clean up of all resources held by {@link FileHandle}.
+ */
+ private static class Cleanup implements RefCounted.Tidy
+ {
+ final ChannelProxy channel;
+ final RebuffererFactory rebufferer;
+ final CompressionMetadata compressionMetadata;
+ final Optional<ChunkCache> chunkCache;
+
+ private Cleanup(ChannelProxy channel,
+ RebuffererFactory rebufferer,
+ CompressionMetadata compressionMetadata,
+ ChunkCache chunkCache)
+ {
+ this.channel = channel;
+ this.rebufferer = rebufferer;
+ this.compressionMetadata = compressionMetadata;
+ this.chunkCache = Optional.ofNullable(chunkCache);
+ }
+
+ public String name()
+ {
+ return channel.filePath();
+ }
+
+ public void tidy()
+ {
+ chunkCache.ifPresent(cache -> cache.invalidateFile(name()));
+ try
+ {
+ if (compressionMetadata != null)
+ {
+ compressionMetadata.close();
+ }
+ }
+ finally
+ {
+ try
+ {
+ channel.close();
+ }
+ finally
+ {
+ rebufferer.close();
+ }
+ }
+ }
+ }
+
+ /**
+ * Configures how the file will be read (compressed, mmapped, use cache etc.)
+ */
+ public static class Builder implements AutoCloseable
+ {
+ private final String path;
+
+ private ChannelProxy channel;
+ private CompressionMetadata compressionMetadata;
+ private MmappedRegions regions;
+ private ChunkCache chunkCache;
+ private int bufferSize = RandomAccessReader.DEFAULT_BUFFER_SIZE;
+ private BufferType bufferType = BufferType.OFF_HEAP;
+
+ private boolean mmapped = false;
+ private boolean compressed = false;
+
+ public Builder(String path)
+ {
+ this.path = path;
+ }
+
+ public Builder(ChannelProxy channel)
+ {
+ this.channel = channel;
+ this.path = channel.filePath();
+ }
+
+ public Builder compressed(boolean compressed)
+ {
+ this.compressed = compressed;
+ return this;
+ }
+
+ /**
+ * Set {@link ChunkCache} to use.
+ *
+ * @param chunkCache ChunkCache object to use for caching
+ * @return this object
+ */
+ public Builder withChunkCache(ChunkCache chunkCache)
+ {
+ this.chunkCache = chunkCache;
+ return this;
+ }
+
+ /**
+ * Provide {@link CompressionMetadata} to use when reading compressed file.
+ *
+ * @param metadata CompressionMetadata to use
+ * @return this object
+ */
+ public Builder withCompressionMetadata(CompressionMetadata metadata)
+ {
+ this.compressed = Objects.nonNull(metadata);
+ this.compressionMetadata = metadata;
+ return this;
+ }
+
+ /**
+ * Set whether to use mmap for reading
+ *
+ * @param mmapped true if using mmap
+ * @return this instance
+ */
+ public Builder mmapped(boolean mmapped)
+ {
+ this.mmapped = mmapped;
+ return this;
+ }
+
+ /**
+ * Set the buffer size to use (if appropriate).
+ *
+ * @param bufferSize Buffer size in bytes
+ * @return this instance
+ */
+ public Builder bufferSize(int bufferSize)
+ {
+ this.bufferSize = bufferSize;
+ return this;
+ }
+
+ /**
+ * Set the buffer type (on heap or off heap) to use (if appropriate).
+ *
+ * @param bufferType Buffer type to use
+ * @return this instance
+ */
+ public Builder bufferType(BufferType bufferType)
+ {
+ this.bufferType = bufferType;
+ return this;
+ }
+
+ /**
+ * Complete building {@link FileHandle} without overriding file length.
+ *
+ * @see #complete(long)
+ */
+ public FileHandle complete()
+ {
+ return complete(-1L);
+ }
+
+ /**
+ * Complete building {@link FileHandle} with the given length, which overrides the file length.
+ *
+ * @param overrideLength Override file length (in bytes) so that read cannot go further than this value.
+ * If the value is less than or equal to 0, then the value is ignored.
+ * @return Built file
+ */
+ @SuppressWarnings("resource")
+ public FileHandle complete(long overrideLength)
+ {
+ if (channel == null)
+ {
+ channel = new ChannelProxy(path);
+ }
+
+ ChannelProxy channelCopy = channel.sharedCopy();
+ try
+ {
+ if (compressed && compressionMetadata == null)
+ compressionMetadata = CompressionMetadata.create(channelCopy.filePath());
+
+ long length = overrideLength > 0 ? overrideLength : compressed ? compressionMetadata.compressedFileLength : channelCopy.size();
+
+ RebuffererFactory rebuffererFactory;
+ if (mmapped)
+ {
+ if (compressed)
+ {
+ regions = MmappedRegions.map(channelCopy, compressionMetadata);
+ rebuffererFactory = maybeCached(new CompressedChunkReader.Mmap(channelCopy, compressionMetadata,
+ regions));
+ }
+ else
+ {
+ updateRegions(channelCopy, length);
+ rebuffererFactory = new MmapRebufferer(channelCopy, length, regions.sharedCopy());
+ }
+ }
+ else
+ {
+ regions = null;
+ if (compressed)
+ {
+ rebuffererFactory = maybeCached(new CompressedChunkReader.Standard(channelCopy, compressionMetadata));
+ }
+ else
+ {
+ rebuffererFactory = maybeCached(new SimpleChunkReader(channelCopy, length, bufferType, bufferSize));
+ }
+ }
+ Cleanup cleanup = new Cleanup(channelCopy, rebuffererFactory, compressionMetadata, chunkCache);
+ return new FileHandle(cleanup, channelCopy, rebuffererFactory, compressionMetadata, length);
+ }
+ catch (Throwable t)
+ {
+ channelCopy.close();
+ throw t;
+ }
+ }
+
+ public Throwable close(Throwable accumulate)
+ {
+ if (!compressed && regions != null)
+ accumulate = regions.close(accumulate);
+ if (channel != null)
+ return channel.close(accumulate);
+
+ return accumulate;
+ }
+
+ public void close()
+ {
+ maybeFail(close(null));
+ }
+
+ private RebuffererFactory maybeCached(ChunkReader reader)
+ {
+ if (chunkCache != null && chunkCache.capacity() > 0)
+ return chunkCache.wrap(reader);
+ return reader;
+ }
+
+ private void updateRegions(ChannelProxy channel, long length)
+ {
+ if (regions != null && !regions.isValid(channel))
+ {
+ Throwable err = regions.close(null);
+ if (err != null)
+ logger.error("Failed to close mapped regions", err);
+
+ regions = null;
+ }
+
+ if (regions == null)
+ regions = MmappedRegions.map(channel, length);
+ else
+ regions.extend(length);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(path='" + path() + '\'' +
+ ", length=" + rebuffererFactory.fileLength() +
+ ')';
+ }
+}
diff --git a/src/java/org/apache/cassandra/io/util/Memory.java b/src/java/org/apache/cassandra/io/util/Memory.java
index 78950ce..bd87f5e 100644
--- a/src/java/org/apache/cassandra/io/util/Memory.java
+++ b/src/java/org/apache/cassandra/io/util/Memory.java
@@ -22,6 +22,8 @@
import java.nio.ByteOrder;
import net.nicoulaj.compilecommand.annotations.Inline;
+
+import org.apache.cassandra.utils.Architecture;
import org.apache.cassandra.utils.FastByteOperations;
import org.apache.cassandra.utils.concurrent.Ref;
import org.apache.cassandra.utils.memory.MemoryUtil;
@@ -51,17 +53,9 @@
private static final long BYTE_ARRAY_BASE_OFFSET = unsafe.arrayBaseOffset(byte[].class);
private static final boolean bigEndian = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
- private static final boolean unaligned;
public static final ByteBuffer[] NO_BYTE_BUFFERS = new ByteBuffer[0];
- static
- {
- String arch = System.getProperty("os.arch");
- unaligned = arch.equals("i386") || arch.equals("x86")
- || arch.equals("amd64") || arch.equals("x86_64") || arch.equals("s390x");
- }
-
protected long peer;
// size of the memory region
protected final long size;
@@ -113,7 +107,7 @@
public void setLong(long offset, long l)
{
checkBounds(offset, offset + 8);
- if (unaligned)
+ if (Architecture.IS_UNALIGNED)
{
unsafe.putLong(peer + offset, l);
}
@@ -152,7 +146,7 @@
public void setInt(long offset, int l)
{
checkBounds(offset, offset + 4);
- if (unaligned)
+ if (Architecture.IS_UNALIGNED)
{
unsafe.putInt(peer + offset, l);
}
@@ -183,7 +177,7 @@
public void setShort(long offset, short l)
{
checkBounds(offset, offset + 2);
- if (unaligned)
+ if (Architecture.IS_UNALIGNED)
{
unsafe.putShort(peer + offset, l);
}
@@ -258,7 +252,7 @@
public long getLong(long offset)
{
checkBounds(offset, offset + 8);
- if (unaligned) {
+ if (Architecture.IS_UNALIGNED) {
return unsafe.getLong(peer + offset);
} else {
return getLongByByte(peer + offset);
@@ -290,7 +284,7 @@
public int getInt(long offset)
{
checkBounds(offset, offset + 4);
- if (unaligned) {
+ if (Architecture.IS_UNALIGNED) {
return unsafe.getInt(peer + offset);
} else {
return getIntByByte(peer + offset);
diff --git a/src/java/org/apache/cassandra/io/util/MmapRebufferer.java b/src/java/org/apache/cassandra/io/util/MmapRebufferer.java
index 9d79919..8df6370 100644
--- a/src/java/org/apache/cassandra/io/util/MmapRebufferer.java
+++ b/src/java/org/apache/cassandra/io/util/MmapRebufferer.java
@@ -28,7 +28,7 @@
{
protected final MmappedRegions regions;
- public MmapRebufferer(ChannelProxy channel, long fileLength, MmappedRegions regions)
+ MmapRebufferer(ChannelProxy channel, long fileLength, MmappedRegions regions)
{
super(channel, fileLength);
this.regions = regions;
diff --git a/src/java/org/apache/cassandra/io/util/MmappedRegions.java b/src/java/org/apache/cassandra/io/util/MmappedRegions.java
index f269b84..9ab8abf 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedRegions.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedRegions.java
@@ -91,6 +91,11 @@
return new MmappedRegions(channel, null, 0);
}
+ /**
+ * @param channel file to map. the MmappedRegions instance will hold shared copy of given channel.
+ * @param metadata
+ * @return new instance
+ */
public static MmappedRegions map(ChannelProxy channel, CompressionMetadata metadata)
{
if (metadata == null)
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
deleted file mode 100644
index d514bf8..0000000
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ /dev/null
@@ -1,117 +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.io.util;
-
-import java.io.*;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.sstable.format.Version;
-
-public class MmappedSegmentedFile extends SegmentedFile
-{
- private static final Logger logger = LoggerFactory.getLogger(MmappedSegmentedFile.class);
-
- public MmappedSegmentedFile(ChannelProxy channel, long length, MmappedRegions regions)
- {
- this(channel, new MmapRebufferer(channel, length, regions), length);
- }
-
- public MmappedSegmentedFile(ChannelProxy channel, RebuffererFactory rebufferer, long length)
- {
- super(new Cleanup(channel, rebufferer), channel, rebufferer, length);
- }
-
- private MmappedSegmentedFile(MmappedSegmentedFile copy)
- {
- super(copy);
- }
-
- public MmappedSegmentedFile sharedCopy()
- {
- return new MmappedSegmentedFile(this);
- }
-
- /**
- * Overrides the default behaviour to create segments of a maximum size.
- */
- static class Builder extends SegmentedFile.Builder
- {
- private MmappedRegions regions;
-
- Builder()
- {
- super();
- }
-
- public SegmentedFile complete(ChannelProxy channel, int bufferSize, long overrideLength)
- {
- long length = overrideLength > 0 ? overrideLength : channel.size();
- updateRegions(channel, length);
-
- return new MmappedSegmentedFile(channel, length, regions.sharedCopy());
- }
-
- private void updateRegions(ChannelProxy channel, long length)
- {
- if (regions != null && !regions.isValid(channel))
- {
- Throwable err = regions.close(null);
- if (err != null)
- logger.error("Failed to close mapped regions", err);
-
- regions = null;
- }
-
- if (regions == null)
- regions = MmappedRegions.map(channel, length);
- else
- regions.extend(length);
- }
-
- @Override
- public void serializeBounds(DataOutput out, Version version) throws IOException
- {
- if (!version.hasBoundaries())
- return;
-
- super.serializeBounds(out, version);
- out.writeInt(0);
- }
-
- @Override
- public void deserializeBounds(DataInput in, Version version) throws IOException
- {
- if (!version.hasBoundaries())
- return;
-
- super.deserializeBounds(in, version);
- in.skipBytes(in.readInt() * TypeSizes.sizeof(0L));
- }
-
- @Override
- public Throwable close(Throwable accumulate)
- {
- return super.close(regions == null
- ? accumulate
- : regions.close(accumulate));
- }
- }
-}
diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
index 725b367..5157eac 100644
--- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
@@ -17,50 +17,37 @@
*/
package org.apache.cassandra.io.util;
-import java.io.*;
-import java.nio.ByteBuffer;
+import java.io.File;
+import java.io.IOException;
import java.nio.ByteOrder;
-import com.google.common.annotations.VisibleForTesting;
import com.google.common.primitives.Ints;
-import com.google.common.util.concurrent.RateLimiter;
import org.apache.cassandra.io.compress.BufferType;
-import org.apache.cassandra.io.compress.CompressionMetadata;
import org.apache.cassandra.io.util.Rebufferer.BufferHolder;
-import org.apache.cassandra.utils.memory.BufferPool;
public class RandomAccessReader extends RebufferingInputStream implements FileDataInput
{
// The default buffer size when the client doesn't specify it
public static final int DEFAULT_BUFFER_SIZE = 4096;
- // The maximum buffer size, we will never buffer more than this size. Further,
- // when the limiter is not null, i.e. when throttling is enabled, we read exactly
- // this size, since when throttling the intention is to eventually read everything,
- // see CASSANDRA-8630
- // NOTE: this size is chosen both for historical consistency, as a reasonable upper bound,
- // and because our BufferPool currently has a maximum allocation size of this.
- public static final int MAX_BUFFER_SIZE = 1 << 16; // 64k
-
// offset of the last file mark
- protected long markedPointer;
+ private long markedPointer;
- @VisibleForTesting
final Rebufferer rebufferer;
- BufferHolder bufferHolder = Rebufferer.EMPTY;
+ private BufferHolder bufferHolder = Rebufferer.EMPTY;
- protected RandomAccessReader(Rebufferer rebufferer)
+ /**
+ * Only created through Builder
+ *
+ * @param rebufferer Rebufferer to use
+ */
+ RandomAccessReader(Rebufferer rebufferer)
{
super(Rebufferer.EMPTY.buffer());
this.rebufferer = rebufferer;
}
- public static ByteBuffer allocateBuffer(int size, BufferType bufferType)
- {
- return BufferPool.get(size, bufferType).order(ByteOrder.BIG_ENDIAN);
- }
-
/**
* Read data from file starting from current currentOffset to populate buffer.
*/
@@ -72,7 +59,7 @@
reBufferAt(current());
}
- public void reBufferAt(long position)
+ private void reBufferAt(long position)
{
bufferHolder.release();
bufferHolder = rebufferer.rebuffer(position);
@@ -188,11 +175,11 @@
/**
* Class to hold a mark to the position of the file
*/
- protected static class BufferedRandomAccessFileMark implements DataPosition
+ private static class BufferedRandomAccessFileMark implements DataPosition
{
final long pointer;
- public BufferedRandomAccessFileMark(long pointer)
+ private BufferedRandomAccessFileMark(long pointer)
{
this.pointer = pointer;
}
@@ -283,139 +270,14 @@
return rebufferer.getCrcCheckChance();
}
- protected static Rebufferer instantiateRebufferer(RebuffererFactory fileRebufferer, RateLimiter limiter)
- {
- Rebufferer rebufferer = fileRebufferer.instantiateRebufferer();
-
- if (limiter != null)
- rebufferer = new LimitingRebufferer(rebufferer, limiter, MAX_BUFFER_SIZE);
-
- return rebufferer;
- }
-
- public static RandomAccessReader build(SegmentedFile file, RateLimiter limiter)
- {
- return new RandomAccessReader(instantiateRebufferer(file.rebuffererFactory(), limiter));
- }
-
- public static Builder builder(ChannelProxy channel)
- {
- return new Builder(channel);
- }
-
- public static class Builder
- {
- // The NIO file channel or an empty channel
- public final ChannelProxy channel;
-
- // The size of the buffer for buffered readers
- protected int bufferSize;
-
- // The type of the buffer for buffered readers
- public BufferType bufferType;
-
- // The buffer
- public ByteBuffer buffer;
-
- // An optional limiter that will throttle the amount of data we read
- public RateLimiter limiter;
-
- // The mmap segments for mmap readers
- public MmappedRegions regions;
-
- // Compression for compressed readers
- public CompressionMetadata compression;
-
- public Builder(ChannelProxy channel)
- {
- this.channel = channel;
- this.bufferSize = DEFAULT_BUFFER_SIZE;
- this.bufferType = BufferType.OFF_HEAP;
- }
-
- /** The buffer size is typically already page aligned but if that is not the case
- * make sure that it is a multiple of the page size, 4096. Also limit it to the maximum
- * buffer size unless we are throttling, in which case we may as well read the maximum
- * directly since the intention is to read the full file, see CASSANDRA-8630.
- * */
- private int adjustedBufferSize()
- {
- if (limiter != null)
- return MAX_BUFFER_SIZE;
-
- // should already be a page size multiple but if that's not case round it up
- int wholePageSize = (bufferSize + 4095) & ~4095;
- return Math.min(MAX_BUFFER_SIZE, wholePageSize);
- }
-
- protected Rebufferer createRebufferer()
- {
- return instantiateRebufferer(chunkReader(), limiter);
- }
-
- public RebuffererFactory chunkReader()
- {
- if (compression != null)
- return CompressedSegmentedFile.chunkReader(channel, compression, regions);
- if (regions != null)
- return new MmapRebufferer(channel, -1, regions);
-
- int adjustedSize = adjustedBufferSize();
- return new SimpleChunkReader(channel, -1, bufferType, adjustedSize);
- }
-
- public Builder bufferSize(int bufferSize)
- {
- if (bufferSize <= 0)
- throw new IllegalArgumentException("bufferSize must be positive");
-
- this.bufferSize = bufferSize;
- return this;
- }
-
- public Builder bufferType(BufferType bufferType)
- {
- this.bufferType = bufferType;
- return this;
- }
-
- public Builder regions(MmappedRegions regions)
- {
- this.regions = regions;
- return this;
- }
-
- public Builder compression(CompressionMetadata metadata)
- {
- this.compression = metadata;
- return this;
- }
-
- public Builder limiter(RateLimiter limiter)
- {
- this.limiter = limiter;
- return this;
- }
-
- public RandomAccessReader build()
- {
- return new RandomAccessReader(createRebufferer());
- }
-
- public RandomAccessReader buildWithChannel()
- {
- return new RandomAccessReaderWithOwnChannel(createRebufferer());
- }
- }
-
// A wrapper of the RandomAccessReader that closes the channel when done.
// For performance reasons RAR does not increase the reference count of
// a channel but assumes the owner will keep it open and close it,
// see CASSANDRA-9379, this thin class is just for those cases where we do
// not have a shared channel.
- public static class RandomAccessReaderWithOwnChannel extends RandomAccessReader
+ static class RandomAccessReaderWithOwnChannel extends RandomAccessReader
{
- protected RandomAccessReaderWithOwnChannel(Rebufferer rebufferer)
+ RandomAccessReaderWithOwnChannel(Rebufferer rebufferer)
{
super(rebufferer);
}
@@ -441,14 +303,26 @@
}
}
+ /**
+ * Open a RandomAccessReader (not compressed, not mmapped, no read throttling) that will own its channel.
+ *
+ * @param file File to open for reading
+ * @return new RandomAccessReader that owns the channel opened in this method.
+ */
@SuppressWarnings("resource")
public static RandomAccessReader open(File file)
{
- return new Builder(new ChannelProxy(file)).buildWithChannel();
- }
-
- public static RandomAccessReader open(ChannelProxy channel)
- {
- return new Builder(channel).build();
+ ChannelProxy channel = new ChannelProxy(file);
+ try
+ {
+ ChunkReader reader = new SimpleChunkReader(channel, -1, BufferType.OFF_HEAP, DEFAULT_BUFFER_SIZE);
+ Rebufferer rebufferer = reader.instantiateRebufferer();
+ return new RandomAccessReaderWithOwnChannel(rebufferer);
+ }
+ catch (Throwable t)
+ {
+ channel.close();
+ throw t;
+ }
}
}
diff --git a/src/java/org/apache/cassandra/io/util/Rebufferer.java b/src/java/org/apache/cassandra/io/util/Rebufferer.java
index e88c7cb..2fc7ffa 100644
--- a/src/java/org/apache/cassandra/io/util/Rebufferer.java
+++ b/src/java/org/apache/cassandra/io/util/Rebufferer.java
@@ -38,7 +38,7 @@
*/
void closeReader();
- public interface BufferHolder
+ interface BufferHolder
{
/**
* Returns a useable buffer (i.e. one whose position and limit can be freely modified). Its limit will be set
@@ -59,7 +59,7 @@
void release();
}
- static final BufferHolder EMPTY = new BufferHolder()
+ BufferHolder EMPTY = new BufferHolder()
{
final ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0);
diff --git a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
index 88912f9..24eb93c 100644
--- a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
@@ -33,7 +33,7 @@
private SafeMemoryWriter(SafeMemory memory)
{
- super(tailBuffer(memory).order(ByteOrder.BIG_ENDIAN), null);
+ super(tailBuffer(memory).order(ByteOrder.BIG_ENDIAN));
this.memory = memory;
}
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
deleted file mode 100644
index 62e14ba..0000000
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ /dev/null
@@ -1,318 +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.io.util;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.File;
-import java.io.IOException;
-
-import com.google.common.util.concurrent.RateLimiter;
-
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.io.compress.CompressedSequentialWriter;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.IndexSummary;
-import org.apache.cassandra.io.sstable.IndexSummaryBuilder;
-import org.apache.cassandra.io.sstable.format.Version;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.utils.CLibrary;
-import org.apache.cassandra.utils.concurrent.RefCounted;
-import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
-
-import static org.apache.cassandra.utils.Throwables.maybeFail;
-
-/**
- * Abstracts a read-only file that has been split into segments, each of which can be represented by an independent
- * FileDataInput. Allows for iteration over the FileDataInputs, or random access to the FileDataInput for a given
- * position.
- *
- * The JVM can only map up to 2GB at a time, so each segment is at most that size when using mmap i/o. If a segment
- * would need to be longer than 2GB, that segment will not be mmap'd, and a new RandomAccessFile will be created for
- * each access to that segment.
- */
-public abstract class SegmentedFile extends SharedCloseableImpl
-{
- public final ChannelProxy channel;
-
- // This differs from length for compressed files (but we still need length for
- // SegmentIterator because offsets in the file are relative to the uncompressed size)
- public final long onDiskLength;
-
- /**
- * Rebufferer to use to construct RandomAccessReaders.
- */
- private final RebuffererFactory rebufferer;
-
- protected SegmentedFile(Cleanup cleanup, ChannelProxy channel, RebuffererFactory rebufferer, long onDiskLength)
- {
- super(cleanup);
- this.rebufferer = rebufferer;
- this.channel = channel;
- this.onDiskLength = onDiskLength;
- }
-
- protected SegmentedFile(SegmentedFile copy)
- {
- super(copy);
- channel = copy.channel;
- rebufferer = copy.rebufferer;
- onDiskLength = copy.onDiskLength;
- }
-
- public String path()
- {
- return channel.filePath();
- }
-
- public long dataLength()
- {
- return rebufferer.fileLength();
- }
-
- public RebuffererFactory rebuffererFactory()
- {
- return rebufferer;
- }
-
- protected static class Cleanup implements RefCounted.Tidy
- {
- final ChannelProxy channel;
- final ReaderFileProxy rebufferer;
- protected Cleanup(ChannelProxy channel, ReaderFileProxy rebufferer)
- {
- this.channel = channel;
- this.rebufferer = rebufferer;
- }
-
- public String name()
- {
- return channel.filePath();
- }
-
- public void tidy()
- {
- try
- {
- channel.close();
- }
- finally
- {
- rebufferer.close();
- }
- }
- }
-
- public abstract SegmentedFile sharedCopy();
-
- public RandomAccessReader createReader()
- {
- return RandomAccessReader.build(this, null);
- }
-
- public RandomAccessReader createReader(RateLimiter limiter)
- {
- return RandomAccessReader.build(this, limiter);
- }
-
- public FileDataInput createReader(long position)
- {
- RandomAccessReader reader = createReader();
- reader.seek(position);
- return reader;
- }
-
- public void dropPageCache(long before)
- {
- CLibrary.trySkipCache(channel.getFileDescriptor(), 0, before, path());
- }
-
- /**
- * @return A SegmentedFile.Builder.
- */
- public static Builder getBuilder(Config.DiskAccessMode mode, boolean compressed)
- {
- return compressed ? new CompressedSegmentedFile.Builder(null)
- : mode == Config.DiskAccessMode.mmap ? new MmappedSegmentedFile.Builder()
- : new BufferedSegmentedFile.Builder();
- }
-
- public static Builder getCompressedBuilder(CompressedSequentialWriter writer)
- {
- return new CompressedSegmentedFile.Builder(writer);
- }
-
- /**
- * Collects potential segmentation points in an underlying file, and builds a SegmentedFile to represent it.
- */
- public static abstract class Builder implements AutoCloseable
- {
- private ChannelProxy channel;
-
- /**
- * Called after all potential boundaries have been added to apply this Builder to a concrete file on disk.
- * @param channel The channel to the file on disk.
- */
- protected abstract SegmentedFile complete(ChannelProxy channel, int bufferSize, long overrideLength);
-
- @SuppressWarnings("resource") // SegmentedFile owns channel
- private SegmentedFile complete(String path, int bufferSize, long overrideLength)
- {
- ChannelProxy channelCopy = getChannel(path);
- try
- {
- return complete(channelCopy, bufferSize, overrideLength);
- }
- catch (Throwable t)
- {
- channelCopy.close();
- throw t;
- }
- }
-
- public SegmentedFile buildData(Descriptor desc, StatsMetadata stats, IndexSummaryBuilder.ReadableBoundary boundary)
- {
- return complete(desc.filenameFor(Component.DATA), bufferSize(stats), boundary.dataLength);
- }
-
- public SegmentedFile buildData(Descriptor desc, StatsMetadata stats)
- {
- return complete(desc.filenameFor(Component.DATA), bufferSize(stats), -1L);
- }
-
- public SegmentedFile buildIndex(Descriptor desc, IndexSummary indexSummary, IndexSummaryBuilder.ReadableBoundary boundary)
- {
- return complete(desc.filenameFor(Component.PRIMARY_INDEX), bufferSize(desc, indexSummary), boundary.indexLength);
- }
-
- public SegmentedFile buildIndex(Descriptor desc, IndexSummary indexSummary)
- {
- return complete(desc.filenameFor(Component.PRIMARY_INDEX), bufferSize(desc, indexSummary), -1L);
- }
-
- private static int bufferSize(StatsMetadata stats)
- {
- return bufferSize(stats.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
- }
-
- private static int bufferSize(Descriptor desc, IndexSummary indexSummary)
- {
- File file = new File(desc.filenameFor(Component.PRIMARY_INDEX));
- return bufferSize(file.length() / indexSummary.size());
- }
-
- /**
- Return the buffer size for a given record size. For spinning disks always add one page.
- For solid state disks only add one page if the chance of crossing to the next page is more
- than a predifined value, @see Config.disk_optimization_page_cross_chance.
- */
- static int bufferSize(long recordSize)
- {
- Config.DiskOptimizationStrategy strategy = DatabaseDescriptor.getDiskOptimizationStrategy();
- if (strategy == Config.DiskOptimizationStrategy.ssd)
- {
- // The crossing probability is calculated assuming a uniform distribution of record
- // start position in a page, so it's the record size modulo the page size divided by
- // the total page size.
- double pageCrossProbability = (recordSize % 4096) / 4096.;
- // if the page cross probability is equal or bigger than disk_optimization_page_cross_chance we add one page
- if ((pageCrossProbability - DatabaseDescriptor.getDiskOptimizationPageCrossChance()) > -1e-16)
- recordSize += 4096;
-
- return roundBufferSize(recordSize);
- }
- else if (strategy == Config.DiskOptimizationStrategy.spinning)
- {
- return roundBufferSize(recordSize + 4096);
- }
- else
- {
- throw new IllegalStateException("Unsupported disk optimization strategy: " + strategy);
- }
- }
-
- /**
- Round up to the next multiple of 4k but no more than 64k
- */
- static int roundBufferSize(long size)
- {
- if (size <= 0)
- return 4096;
-
- size = (size + 4095) & ~4095;
- return (int)Math.min(size, 1 << 16);
- }
-
- public void serializeBounds(DataOutput out, Version version) throws IOException
- {
- if (!version.hasBoundaries())
- return;
-
- out.writeUTF(DatabaseDescriptor.getDiskAccessMode().name());
- }
-
- public void deserializeBounds(DataInput in, Version version) throws IOException
- {
- if (!version.hasBoundaries())
- return;
-
- if (!in.readUTF().equals(DatabaseDescriptor.getDiskAccessMode().name()))
- throw new IOException("Cannot deserialize SSTable Summary component because the DiskAccessMode was changed!");
- }
-
- public Throwable close(Throwable accumulate)
- {
- if (channel != null)
- return channel.close(accumulate);
-
- return accumulate;
- }
-
- public void close()
- {
- maybeFail(close(null));
- }
-
- private ChannelProxy getChannel(String path)
- {
- if (channel != null)
- {
- // This is really fragile, both path and channel.filePath()
- // must agree, i.e. they both must be absolute or both relative
- // eventually we should really pass the filePath to the builder
- // constructor and remove this
- if (channel.filePath().equals(path))
- return channel.sharedCopy();
- else
- channel.close();
- }
-
- channel = new ChannelProxy(path);
- return channel.sharedCopy();
- }
- }
-
- @Override
- public String toString() {
- return getClass().getSimpleName() + "(path='" + path() + '\'' +
- ", length=" + rebufferer.fileLength() +
- ')';
- }
-}
diff --git a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java
index 7bfb57b..bc1a529 100644
--- a/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java
+++ b/src/java/org/apache/cassandra/io/util/SimpleChunkReader.java
@@ -27,7 +27,7 @@
private final int bufferSize;
private final BufferType bufferType;
- public SimpleChunkReader(ChannelProxy channel, long fileLength, BufferType bufferType, int bufferSize)
+ SimpleChunkReader(ChannelProxy channel, long fileLength, BufferType bufferType, int bufferSize)
{
super(channel, fileLength);
this.bufferSize = bufferSize;
@@ -55,15 +55,9 @@
}
@Override
- public boolean alignmentRequired()
- {
- return false;
- }
-
- @Override
public Rebufferer instantiateRebufferer()
{
- return BufferManagingRebufferer.on(this);
+ return new BufferManagingRebufferer.Unaligned(this);
}
@Override
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/src/java/org/apache/cassandra/io/util/SpinningDiskOptimizationStrategy.java
similarity index 76%
copy from src/java/org/apache/cassandra/io/util/ICompressedFile.java
copy to src/java/org/apache/cassandra/io/util/SpinningDiskOptimizationStrategy.java
index e69487c..5cec282 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SpinningDiskOptimizationStrategy.java
@@ -15,12 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.compress.CompressionMetadata;
-
-public interface ICompressedFile
+public class SpinningDiskOptimizationStrategy implements DiskOptimizationStrategy
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ /**
+ * For spinning disks always add one page.
+ */
+ @Override
+ public int bufferSize(long recordSize)
+ {
+ return roundBufferSize(recordSize + 4096);
+ }
}
diff --git a/src/java/org/apache/cassandra/io/util/SsdDiskOptimizationStrategy.java b/src/java/org/apache/cassandra/io/util/SsdDiskOptimizationStrategy.java
new file mode 100644
index 0000000..032ec2b
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/SsdDiskOptimizationStrategy.java
@@ -0,0 +1,49 @@
+/*
+ * 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.io.util;
+
+public class SsdDiskOptimizationStrategy implements DiskOptimizationStrategy
+{
+ private final double diskOptimizationPageCrossChance;
+
+ public SsdDiskOptimizationStrategy(double diskOptimizationPageCrossChance)
+ {
+ this.diskOptimizationPageCrossChance = diskOptimizationPageCrossChance;
+ }
+
+ /**
+ * For solid state disks only add one page if the chance of crossing to the next page is more
+ * than a predifined value.
+ *
+ * @see org.apache.cassandra.config.Config#disk_optimization_page_cross_chance
+ */
+ @Override
+ public int bufferSize(long recordSize)
+ {
+ // The crossing probability is calculated assuming a uniform distribution of record
+ // start position in a page, so it's the record size modulo the page size divided by
+ // the total page size.
+ double pageCrossProbability = (recordSize % 4096) / 4096.;
+ // if the page cross probability is equal or bigger than disk_optimization_page_cross_chance we add one page
+ if ((pageCrossProbability - diskOptimizationPageCrossChance) > -1e-16)
+ recordSize += 4096;
+
+ return roundBufferSize(recordSize);
+ }
+}
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
index 70aecb0..975b10e 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
@@ -31,6 +31,10 @@
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.ApplicationState;
+import org.apache.cassandra.gms.EndpointState;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.gms.VersionedValue;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.FBUtilities;
@@ -283,7 +287,7 @@
// finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
// "Severity" is basically a measure of compaction activity (CASSANDRA-3722).
if (USE_SEVERITY)
- score += StorageService.instance.getSeverity(entry.getKey());
+ score += getSeverity(entry.getKey());
// lowest score (least amount of badness) wins.
newScores.put(entry.getKey(), score);
}
@@ -333,12 +337,25 @@
public void setSeverity(double severity)
{
- StorageService.instance.reportManualSeverity(severity);
+ Gossiper.instance.addLocalApplicationState(ApplicationState.SEVERITY, StorageService.instance.valueFactory.severity(severity));
+ }
+
+ private double getSeverity(InetAddress endpoint)
+ {
+ EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
+ if (state == null)
+ return 0.0;
+
+ VersionedValue event = state.getApplicationState(ApplicationState.SEVERITY);
+ if (event == null)
+ return 0.0;
+
+ return Double.parseDouble(event.value);
}
public double getSeverity()
{
- return StorageService.instance.getSeverity(FBUtilities.getBroadcastAddress());
+ return getSeverity(FBUtilities.getBroadcastAddress());
}
public boolean isWorthMergingForRangeQuery(List<InetAddress> merged, List<InetAddress> l1, List<InetAddress> l2)
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java
index a413bc5..552a16d 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitchMBean.java
@@ -29,11 +29,30 @@
public double getBadnessThreshold();
public String getSubsnitchClassName();
public List<Double> dumpTimings(String hostname) throws UnknownHostException;
+
/**
- * Use this if you want to specify a severity; it can be negative
- * Example: Page cache is cold and you want data to be sent
- * though it is not preferred one.
+ * Setting a Severity allows operators to inject preference information into the Dynamic Snitch
+ * replica selection.
+ *
+ * When choosing which replicas to participate in a read request, the DSnitch sorts replicas
+ * by response latency, and selects the fastest replicas. Latencies are normalized to a score
+ * from 0 to 1, with lower scores being faster.
+ *
+ * The Severity injected here will be added to the normalized score.
+ *
+ * Thus, adding a Severity greater than 1 will mean the replica will never be contacted
+ * (unless needed for ALL or if it is added later for rapid read protection).
+ *
+ * Conversely, adding a negative Severity means the replica will *always* be contacted.
+ *
+ * (The "Severity" term is historical and dates to when this was used to represent how
+ * badly background tasks like compaction were affecting a replica's performance.
+ * See CASSANDRA-3722 for when this was introduced and CASSANDRA-11738 for why it was removed.)
*/
public void setSeverity(double severity);
+
+ /**
+ * @return the current manually injected Severity.
+ */
public double getSeverity();
}
diff --git a/src/java/org/apache/cassandra/metrics/AuthMetrics.java b/src/java/org/apache/cassandra/metrics/AuthMetrics.java
new file mode 100644
index 0000000..126738c
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/AuthMetrics.java
@@ -0,0 +1,40 @@
+package org.apache.cassandra.metrics;
+
+import com.codahale.metrics.Meter;
+
+/**
+ * Metrics about authentication
+ */
+public class AuthMetrics
+{
+
+ public static final AuthMetrics instance = new AuthMetrics();
+
+ public static void init()
+ {
+ // no-op, just used to force instance creation
+ }
+
+ /** Number and rate of successful logins */
+ protected final Meter success;
+
+ /** Number and rate of login failures */
+ protected final Meter failure;
+
+ private AuthMetrics()
+ {
+
+ success = ClientMetrics.instance.addMeter("AuthSuccess");
+ failure = ClientMetrics.instance.addMeter("AuthFailure");
+ }
+
+ public void markSuccess()
+ {
+ success.mark();
+ }
+
+ public void markFailure()
+ {
+ failure.mark();
+ }
+}
diff --git a/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java b/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java
index 107717d..c9c859a 100644
--- a/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java
@@ -19,7 +19,6 @@
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Meter;
-import com.codahale.metrics.RatioGauge;
import org.apache.cassandra.utils.memory.BufferPool;
import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
diff --git a/src/java/org/apache/cassandra/metrics/ClientMetrics.java b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
index 4a384eb..db6422c 100644
--- a/src/java/org/apache/cassandra/metrics/ClientMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
@@ -21,6 +21,7 @@
import java.util.concurrent.Callable;
import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
@@ -37,18 +38,19 @@
public void addCounter(String name, final Callable<Integer> provider)
{
- Metrics.register(factory.createMetricName(name), new Gauge<Integer>()
- {
- public Integer getValue()
+ Metrics.register(factory.createMetricName(name), (Gauge<Integer>) () -> {
+ try
{
- try
- {
- return provider.call();
- } catch (Exception e)
- {
- throw new RuntimeException(e);
- }
+ return provider.call();
+ } catch (Exception e)
+ {
+ throw new RuntimeException(e);
}
});
}
+
+ public Meter addMeter(String name)
+ {
+ return Metrics.meter(factory.createMetricName(name));
+ }
}
diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
index 9d2863f..2cddfff 100644
--- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
@@ -140,15 +140,11 @@
public void beginCompaction(CompactionInfo.Holder ci)
{
- // notify
- ci.started();
compactions.add(ci);
}
public void finishCompaction(CompactionInfo.Holder ci)
{
- // notify
- ci.finished();
compactions.remove(ci);
bytesCompacted.inc(ci.getCompactionInfo().getTotal());
totalCompactionsCompleted.mark();
diff --git a/src/java/org/apache/cassandra/net/IVerbHandler.java b/src/java/org/apache/cassandra/net/IVerbHandler.java
index b9f1a54..0995a68 100644
--- a/src/java/org/apache/cassandra/net/IVerbHandler.java
+++ b/src/java/org/apache/cassandra/net/IVerbHandler.java
@@ -19,8 +19,6 @@
import java.io.IOException;
-import org.apache.cassandra.db.ReadCommand;
-
/**
* IVerbHandler provides the method that all verb handlers need to implement.
* The concrete implementation of this interface would provide the functionality
diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
index 9e8e2e1..02147ef 100644
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
@@ -29,6 +29,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import io.netty.util.concurrent.FastThreadLocalThread;
import net.jpountz.lz4.LZ4BlockInputStream;
import net.jpountz.lz4.LZ4FastDecompressor;
import net.jpountz.lz4.LZ4Factory;
@@ -42,7 +43,7 @@
import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
import org.apache.cassandra.io.util.NIODataInputStream;
-public class IncomingTcpConnection extends Thread implements Closeable
+public class IncomingTcpConnection extends FastThreadLocalThread implements Closeable
{
private static final Logger logger = LoggerFactory.getLogger(IncomingTcpConnection.class);
diff --git a/src/java/org/apache/cassandra/net/MessageIn.java b/src/java/org/apache/cassandra/net/MessageIn.java
index 014ee93..df1b4e1 100644
--- a/src/java/org/apache/cassandra/net/MessageIn.java
+++ b/src/java/org/apache/cassandra/net/MessageIn.java
@@ -27,10 +27,8 @@
import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.monitoring.ConstructionTime;
-import org.apache.cassandra.db.monitoring.MonitorableImpl;
import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.FileUtils;
public class MessageIn<T>
{
@@ -75,7 +73,7 @@
{
InetAddress from = CompactEndpointSerializationHelper.deserialize(in);
- MessagingService.Verb verb = MessagingService.Verb.values()[in.readInt()];
+ MessagingService.Verb verb = MessagingService.verbValues[in.readInt()];
int parameterCount = in.readInt();
Map<String, byte[]> parameters;
if (parameterCount == 0)
diff --git a/src/java/org/apache/cassandra/net/MessageOut.java b/src/java/org/apache/cassandra/net/MessageOut.java
index bc5c41b..08d858b 100644
--- a/src/java/org/apache/cassandra/net/MessageOut.java
+++ b/src/java/org/apache/cassandra/net/MessageOut.java
@@ -30,6 +30,7 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.TypeSizes;
import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.FBUtilities;
@@ -113,11 +114,26 @@
out.write(entry.getValue());
}
- long longSize = payloadSize(version);
- assert longSize <= Integer.MAX_VALUE; // larger values are supported in sstables but not messages
- out.writeInt((int) longSize);
if (payload != null)
- serializer.serialize(payload, out, version);
+ {
+ DataOutputBuffer dob = DataOutputBuffer.scratchBuffer.get();
+ try
+ {
+ serializer.serialize(payload, dob, version);
+
+ int size = dob.getLength();
+ out.writeInt(size);
+ out.write(dob.getData(), 0, size);
+ }
+ finally
+ {
+ dob.recycle();
+ }
+ }
+ else
+ {
+ out.writeInt(0);
+ }
}
public int serializedSize(int version)
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 220fc66..b1f88ee 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -153,6 +153,8 @@
;
}
+ public static final Verb[] verbValues = Verb.values();
+
public static final EnumMap<MessagingService.Verb, Stage> verbStages = new EnumMap<MessagingService.Verb, Stage>(MessagingService.Verb.class)
{{
put(Verb.MUTATION, Stage.MUTATION);
@@ -353,6 +355,11 @@
messageSinks.add(sink);
}
+ public void removeMessageSink(IMessageSink sink)
+ {
+ messageSinks.remove(sink);
+ }
+
public void clearMessageSinks()
{
messageSinks.clear();
diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
index c2d10fd..76b2854 100644
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
@@ -40,6 +40,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import io.netty.util.concurrent.FastThreadLocalThread;
import net.jpountz.lz4.LZ4BlockOutputStream;
import net.jpountz.lz4.LZ4Compressor;
import net.jpountz.lz4.LZ4Factory;
@@ -63,7 +64,7 @@
import com.google.common.util.concurrent.Uninterruptibles;
-public class OutboundTcpConnection extends Thread
+public class OutboundTcpConnection extends FastThreadLocalThread
{
private static final Logger logger = LoggerFactory.getLogger(OutboundTcpConnection.class);
diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
index 312daed..52625bf 100644
--- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
+++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
@@ -21,7 +21,6 @@
import java.util.*;
import com.google.common.base.Predicate;
-import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;
import org.slf4j.Logger;
@@ -30,8 +29,6 @@
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.dht.Bounds;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.net.IVerbHandler;
import org.apache.cassandra.net.MessageIn;
diff --git a/src/java/org/apache/cassandra/repair/ValidationTask.java b/src/java/org/apache/cassandra/repair/ValidationTask.java
index bd866d2..177ad3e 100644
--- a/src/java/org/apache/cassandra/repair/ValidationTask.java
+++ b/src/java/org/apache/cassandra/repair/ValidationTask.java
@@ -18,16 +18,12 @@
package org.apache.cassandra.repair;
import java.net.InetAddress;
-import java.util.Map;
import com.google.common.util.concurrent.AbstractFuture;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.RepairException;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.repair.messages.ValidationRequest;
-import org.apache.cassandra.utils.MerkleTree;
import org.apache.cassandra.utils.MerkleTrees;
/**
diff --git a/src/java/org/apache/cassandra/scheduler/WeightedQueue.java b/src/java/org/apache/cassandra/scheduler/WeightedQueue.java
index 298938d..76c7e9d 100644
--- a/src/java/org/apache/cassandra/scheduler/WeightedQueue.java
+++ b/src/java/org/apache/cassandra/scheduler/WeightedQueue.java
@@ -20,9 +20,6 @@
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.TimeUnit;
-import java.lang.management.ManagementFactory;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
import org.apache.cassandra.metrics.LatencyMetrics;
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
index 93591f0..c70161e 100644
--- a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java
@@ -40,9 +40,7 @@
import org.apache.cassandra.db.rows.RowIterator;
import org.apache.cassandra.db.rows.UnfilteredRowIterators;
import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.OpOrder;
import static java.lang.String.format;
import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
diff --git a/src/java/org/apache/cassandra/schema/Types.java b/src/java/org/apache/cassandra/schema/Types.java
index 25efd70..c2d8aac 100644
--- a/src/java/org/apache/cassandra/schema/Types.java
+++ b/src/java/org/apache/cassandra/schema/Types.java
@@ -29,7 +29,6 @@
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.UserType;
import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.utils.ByteBufferUtil;
import static java.lang.String.format;
import static com.google.common.collect.Iterables.filter;
diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
index e3ba66e..19b3de0 100644
--- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
@@ -31,7 +31,6 @@
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.WriteType;
import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.net.IAsyncCallback;
import org.apache.cassandra.net.IAsyncCallbackWithFailure;
import org.apache.cassandra.net.MessageIn;
import org.apache.cassandra.utils.concurrent.SimpleCondition;
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 04e39db..1ef5dfe 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.service;
-import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
@@ -64,6 +63,7 @@
import org.apache.cassandra.repair.RepairSession;
import org.apache.cassandra.repair.messages.*;
import org.apache.cassandra.utils.CassandraVersion;
+import org.apache.cassandra.utils.Clock;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.UUIDGen;
import org.apache.cassandra.utils.concurrent.Ref;
@@ -260,7 +260,7 @@
public synchronized UUID prepareForRepair(UUID parentRepairSession, InetAddress coordinator, Set<InetAddress> endpoints, RepairOption options, List<ColumnFamilyStore> columnFamilyStores)
{
- long timestamp = System.currentTimeMillis();
+ long timestamp = Clock.instance.currentTimeMillis();
registerParentRepairSession(parentRepairSession, coordinator, columnFamilyStores, options.getRanges(), options.isIncremental(), timestamp, options.isGlobal());
final CountDownLatch prepareLatch = new CountDownLatch(endpoints.size());
final AtomicBoolean status = new AtomicBoolean(true);
diff --git a/src/java/org/apache/cassandra/service/AsyncRepairCallback.java b/src/java/org/apache/cassandra/service/AsyncRepairCallback.java
index 4e70d56..e3ad44e 100644
--- a/src/java/org/apache/cassandra/service/AsyncRepairCallback.java
+++ b/src/java/org/apache/cassandra/service/AsyncRepairCallback.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.service;
-import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.cassandra.concurrent.Stage;
diff --git a/src/java/org/apache/cassandra/service/CacheService.java b/src/java/org/apache/cassandra/service/CacheService.java
index 625d687..a4e18c0 100644
--- a/src/java/org/apache/cassandra/service/CacheService.java
+++ b/src/java/org/apache/cassandra/service/CacheService.java
@@ -54,7 +54,6 @@
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.concurrent.OpOrder;
public class CacheService implements CacheServiceMBean
{
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 0151208..7d82cc7 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -48,6 +48,7 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.functions.ThreadAwareSecurityManager;
+import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.commitlog.CommitLog;
import org.apache.cassandra.exceptions.ConfigurationException;
@@ -343,6 +344,9 @@
SystemKeyspace.finishStartup();
+ // Prepared statements
+ QueryProcessor.preloadPreparedStatement();
+
// Metrics
String metricsReporterConfigFile = System.getProperty("cassandra.metricsReporterConfigFile");
if (metricsReporterConfigFile != null)
diff --git a/src/java/org/apache/cassandra/service/ClientWarn.java b/src/java/org/apache/cassandra/service/ClientWarn.java
index ddad197..878b5e9 100644
--- a/src/java/org/apache/cassandra/service/ClientWarn.java
+++ b/src/java/org/apache/cassandra/service/ClientWarn.java
@@ -20,13 +20,14 @@
import java.util.ArrayList;
import java.util.List;
+import io.netty.util.concurrent.FastThreadLocal;
import org.apache.cassandra.concurrent.ExecutorLocal;
import org.apache.cassandra.utils.FBUtilities;
public class ClientWarn implements ExecutorLocal<ClientWarn.State>
{
private static final String TRUNCATED = " [truncated]";
- private static final ThreadLocal<ClientWarn.State> warnLocal = new ThreadLocal<>();
+ private static final FastThreadLocal<State> warnLocal = new FastThreadLocal<>();
public static ClientWarn instance = new ClientWarn();
private ClientWarn()
diff --git a/src/java/org/apache/cassandra/service/MigrationTask.java b/src/java/org/apache/cassandra/service/MigrationTask.java
index 39a5a11..052b89e 100644
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ b/src/java/org/apache/cassandra/service/MigrationTask.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.service;
-import java.io.IOException;
import java.net.InetAddress;
import java.util.Collection;
import java.util.EnumSet;
diff --git a/src/java/org/apache/cassandra/service/NativeTransportService.java b/src/java/org/apache/cassandra/service/NativeTransportService.java
index 48839f1..d63b2b6 100644
--- a/src/java/org/apache/cassandra/service/NativeTransportService.java
+++ b/src/java/org/apache/cassandra/service/NativeTransportService.java
@@ -32,8 +32,8 @@
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.util.concurrent.EventExecutor;
-import io.netty.util.concurrent.Future;
import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.metrics.AuthMetrics;
import org.apache.cassandra.metrics.ClientMetrics;
import org.apache.cassandra.transport.RequestThreadPoolExecutor;
import org.apache.cassandra.transport.Server;
@@ -116,6 +116,8 @@
return ret;
});
+ AuthMetrics.init();
+
initialized = true;
}
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 3ce8013..2839259 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -73,8 +73,6 @@
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.AbstractIterator;
-import static com.google.common.collect.Iterables.contains;
-
public class StorageProxy implements StorageProxyMBean
{
public static final String MBEAN_NAME = "org.apache.cassandra.db:type=StorageProxy";
@@ -2689,4 +2687,9 @@
public long getReadRepairRepairedBackground() {
return ReadRepairMetrics.repairedBackground.getCount();
}
+
+ public int getNumberOfTables()
+ {
+ return Schema.instance.getNumberOfTables();
+ }
}
diff --git a/src/java/org/apache/cassandra/service/StorageProxyMBean.java b/src/java/org/apache/cassandra/service/StorageProxyMBean.java
index 0db0ca6..0a4ba19 100644
--- a/src/java/org/apache/cassandra/service/StorageProxyMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageProxyMBean.java
@@ -61,4 +61,6 @@
/** Returns each live node's schema version */
public Map<String, List<String>> getSchemaVersions();
+
+ public int getNumberOfTables();
}
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 7500593..d64fc04 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -189,8 +189,6 @@
private final List<IEndpointLifecycleSubscriber> lifecycleSubscribers = new CopyOnWriteArrayList<>();
- private static final BackgroundActivityMonitor bgMonitor = new BackgroundActivityMonitor();
-
private final ObjectName jmxObjectName;
private Collection<Token> bootstrapTokens = null;
@@ -1468,24 +1466,6 @@
}
/**
- * Increment about the known Compaction severity of the events in this node
- */
- public void reportSeverity(double incr)
- {
- bgMonitor.incrCompactionSeverity(incr);
- }
-
- public void reportManualSeverity(double incr)
- {
- bgMonitor.incrManualSeverity(incr);
- }
-
- public double getSeverity(InetAddress endpoint)
- {
- return bgMonitor.getSeverity(endpoint);
- }
-
- /**
* for a keyspace, return the ranges and corresponding listen addresses.
* @param keyspace
* @return the endpoint map
@@ -1895,7 +1875,7 @@
{
try
{
- MessagingService.instance().setVersion(endpoint, Integer.valueOf(value.value));
+ MessagingService.instance().setVersion(endpoint, Integer.parseInt(value.value));
}
catch (NumberFormatException e)
{
diff --git a/src/java/org/apache/cassandra/service/paxos/Commit.java b/src/java/org/apache/cassandra/service/paxos/Commit.java
index 95bd464..af94869 100644
--- a/src/java/org/apache/cassandra/service/paxos/Commit.java
+++ b/src/java/org/apache/cassandra/service/paxos/Commit.java
@@ -35,7 +35,6 @@
import org.apache.cassandra.io.util.DataInputPlus;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.UUIDGen;
import org.apache.cassandra.utils.UUIDSerializer;
diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
index ff81803..544403a 100644
--- a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
@@ -22,7 +22,6 @@
import java.net.InetAddress;
-import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
diff --git a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
index 018dab9..b0bd163 100644
--- a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
@@ -59,7 +59,7 @@
public void response(MessageIn<Boolean> msg)
{
- logger.debug("Propose response {} from {}", msg.payload, msg.from);
+ logger.trace("Propose response {} from {}", msg.payload, msg.from);
if (msg.payload)
accepts.incrementAndGet();
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 88238bc..8fe5a49 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -136,7 +136,7 @@
public synchronized LifecycleTransaction getTransaction()
{
if (done)
- throw new RuntimeException(String.format("Stream receive task {} of cf {} already finished.", session.planId(), cfId));
+ throw new RuntimeException(String.format("Stream receive task %s of cf %s already finished.", session.planId(), cfId));
return txn;
}
diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
index 71ca9b1..61a1c8c 100644
--- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -19,7 +19,6 @@
import java.io.IOException;
import java.net.InetAddress;
-import java.net.Socket;
import java.util.*;
import java.util.concurrent.ConcurrentLinkedQueue;
diff --git a/src/java/org/apache/cassandra/streaming/StreamWriter.java b/src/java/org/apache/cassandra/streaming/StreamWriter.java
index 1d30419..6c86c8b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamWriter.java
+++ b/src/java/org/apache/cassandra/streaming/StreamWriter.java
@@ -32,7 +32,6 @@
import org.apache.cassandra.io.util.DataIntegrityMetadata;
import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.RandomAccessReader;
import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
import org.apache.cassandra.utils.FBUtilities;
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
index 900c1ad..185ab22 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
@@ -18,13 +18,10 @@
package org.apache.cassandra.streaming.compress;
import java.io.IOException;
-import java.nio.channels.WritableByteChannel;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
-import com.google.common.base.Function;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -33,8 +30,6 @@
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.ChannelProxy;
import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
import org.apache.cassandra.streaming.ProgressInfo;
import org.apache.cassandra.streaming.StreamSession;
import org.apache.cassandra.streaming.StreamWriter;
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
index 2b5047d..ec68f9e 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -25,7 +25,6 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.compress.CompressionMetadata;
import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.util.DataInputPlus;
diff --git a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
index 9530e14..dc2eb8f 100644
--- a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
@@ -21,7 +21,6 @@
import java.nio.channels.ReadableByteChannel;
import java.util.List;
-import org.apache.cassandra.io.compress.CompressionMetadata;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.DataOutputStreamPlus;
import org.apache.cassandra.streaming.StreamSession;
diff --git a/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java b/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
index 3603ad5..7d22507 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
@@ -19,7 +19,6 @@
import java.net.InetSocketAddress;
import java.net.SocketAddress;
-import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
diff --git a/src/java/org/apache/cassandra/tools/JsonTransformer.java b/src/java/org/apache/cassandra/tools/JsonTransformer.java
index 3b98595..72008ab 100644
--- a/src/java/org/apache/cassandra/tools/JsonTransformer.java
+++ b/src/java/org/apache/cassandra/tools/JsonTransformer.java
@@ -176,7 +176,6 @@
private void serializePartition(UnfilteredRowIterator partition)
{
- String key = metadata.getKeyValidator().getString(partition.partitionKey().getKey());
try
{
json.writeStartObject();
@@ -223,6 +222,7 @@
}
catch (IOException e)
{
+ String key = metadata.getKeyValidator().getString(partition.partitionKey().getKey());
logger.error("Fatal error parsing partition: {}", key, e);
}
}
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 3bf99ef..84eeb04 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -1203,6 +1203,11 @@
return ThreadPoolMetrics.getJmxThreadPools(mbeanServerConn);
}
+ public int getNumberOfTables()
+ {
+ return spProxy.getNumberOfTables();
+ }
+
/**
* Retrieve ColumnFamily metrics
* @param ks Keyspace for which stats are to be displayed or null for the global value
diff --git a/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java b/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
index 3e2ff08..1f407cb 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
@@ -32,7 +32,6 @@
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.format.SSTableReader;
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index cc6b84b..f05898f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -37,14 +37,12 @@
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.index.SecondaryIndexManager;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.ISSTableScanner;
import org.apache.cassandra.io.sstable.KeyIterator;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
import org.apache.cassandra.utils.FBUtilities;
/**
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 3c8ba64..b455ad7 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -112,8 +112,8 @@
out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
out.printf("SSTable Level: %d%n", stats.sstableLevel);
out.printf("Repaired at: %d%n", stats.repairedAt);
- out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
- out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
+ out.printf("Minimum replay position: %s%n", stats.commitLogLowerBound);
+ out.printf("Maximum replay position: %s%n", stats.commitLogUpperBound);
out.printf("totalColumnsSet: %s%n", stats.totalColumnsSet);
out.printf("totalRows: %s%n", stats.totalRows);
out.println("Estimated tombstone drop times:");
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
index ec729a5..b67f7c4 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
@@ -18,22 +18,13 @@
package org.apache.cassandra.tools.nodetool;
import java.util.*;
-import javax.management.InstanceNotFoundException;
-
-import com.google.common.collect.ArrayListMultimap;
import io.airlift.command.Arguments;
import io.airlift.command.Command;
import io.airlift.command.Option;
-import org.apache.cassandra.db.ColumnFamilyStoreMBean;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.metrics.CassandraMetricsRegistry;
import org.apache.cassandra.tools.NodeProbe;
import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
-import org.apache.cassandra.tools.nodetool.stats.StatsHolder;
-import org.apache.cassandra.tools.nodetool.stats.StatsKeyspace;
-import org.apache.cassandra.tools.nodetool.stats.StatsTable;
-import org.apache.cassandra.tools.nodetool.stats.TableStatsPrinter;
+import org.apache.cassandra.tools.nodetool.stats.*;
@Command(name = "tablestats", description = "Print statistics on tables")
public class TableStats extends NodeToolCmd
@@ -62,260 +53,10 @@
throw new IllegalArgumentException("arguments for -F are json,yaml only.");
}
- TableStats.OptionFilter filter = new OptionFilter(ignore, tableNames);
- ArrayListMultimap<String, ColumnFamilyStoreMBean> selectedTableMbeans = ArrayListMultimap.create();
- Map<String, StatsKeyspace> keyspaceStats = new HashMap<>();
-
- // get a list of table stores
- Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tableMBeans = probe.getColumnFamilyStoreMBeanProxies();
-
- while (tableMBeans.hasNext())
- {
- Map.Entry<String, ColumnFamilyStoreMBean> entry = tableMBeans.next();
- String keyspaceName = entry.getKey();
- ColumnFamilyStoreMBean tableProxy = entry.getValue();
-
- if (filter.isKeyspaceIncluded(keyspaceName))
- {
- StatsKeyspace stats = keyspaceStats.get(keyspaceName);
- if (stats == null)
- {
- stats = new StatsKeyspace(probe, keyspaceName);
- keyspaceStats.put(keyspaceName, stats);
- }
- stats.add(tableProxy);
-
- if (filter.isTableIncluded(keyspaceName, tableProxy.getTableName()))
- selectedTableMbeans.put(keyspaceName, tableProxy);
- }
- }
-
- // make sure all specified keyspace and tables exist
- filter.verifyKeyspaces(probe.getKeyspaces());
- filter.verifyTables();
-
- // get metrics of keyspace
- StatsHolder holder = new StatsHolder();
- for (Map.Entry<String, Collection<ColumnFamilyStoreMBean>> entry : selectedTableMbeans.asMap().entrySet())
- {
- String keyspaceName = entry.getKey();
- Collection<ColumnFamilyStoreMBean> tables = entry.getValue();
- StatsKeyspace statsKeyspace = keyspaceStats.get(keyspaceName);
-
- // get metrics of table statistics for this keyspace
- for (ColumnFamilyStoreMBean table : tables)
- {
- String tableName = table.getTableName();
- StatsTable statsTable = new StatsTable();
- statsTable.name = tableName;
- statsTable.isIndex = tableName.contains(".");
- statsTable.sstableCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveSSTableCount");
- int[] leveledSStables = table.getSSTableCountPerLevel();
- if (leveledSStables != null)
- {
- statsTable.isLeveledSstable = true;
-
- for (int level = 0; level < leveledSStables.length; level++)
- {
- int count = leveledSStables[level];
- long maxCount = 4L; // for L0
- if (level > 0)
- maxCount = (long) Math.pow(10, level);
- // show max threshold for level when exceeded
- statsTable.sstablesInEachLevel.add(count + ((count > maxCount) ? "/" + maxCount : ""));
- }
- }
-
- Long memtableOffHeapSize = null;
- Long bloomFilterOffHeapSize = null;
- Long indexSummaryOffHeapSize = null;
- Long compressionMetadataOffHeapSize = null;
- Long offHeapSize = null;
- Double percentRepaired = null;
-
- try
- {
- memtableOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableOffHeapSize");
- bloomFilterOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterOffHeapMemoryUsed");
- indexSummaryOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "IndexSummaryOffHeapMemoryUsed");
- compressionMetadataOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionMetadataOffHeapMemoryUsed");
- offHeapSize = memtableOffHeapSize + bloomFilterOffHeapSize + indexSummaryOffHeapSize + compressionMetadataOffHeapSize;
- percentRepaired = (Double) probe.getColumnFamilyMetric(keyspaceName, tableName, "PercentRepaired");
- }
- catch (RuntimeException e)
- {
- // offheap-metrics introduced in 2.1.3 - older versions do not have the appropriate mbeans
- if (!(e.getCause() instanceof InstanceNotFoundException))
- throw e;
- }
-
- statsTable.spaceUsedLive = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), humanReadable);
- statsTable.spaceUsedTotal = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), humanReadable);
- statsTable.spaceUsedBySnapshotsTotal = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), humanReadable);
- if (offHeapSize != null)
- {
- statsTable.offHeapUsed = true;
- statsTable.offHeapMemoryUsedTotal = format(offHeapSize, humanReadable);
-
- }
- if (percentRepaired != null)
- {
- statsTable.percentRepaired = Math.round(100 * percentRepaired) / 100.0;
- }
- statsTable.sstableCompressionRatio = probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionRatio");
- Object estimatedPartitionCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "EstimatedPartitionCount");
- if (Long.valueOf(-1L).equals(estimatedPartitionCount))
- {
- estimatedPartitionCount = 0L;
- }
- statsTable.numberOfKeysEstimate = estimatedPartitionCount;
-
- statsTable.memtableCellCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableColumnsCount");
- statsTable.memtableDataSize = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), humanReadable);
- if (memtableOffHeapSize != null)
- {
- statsTable.memtableOffHeapUsed = true;
- statsTable.memtableOffHeapMemoryUsed = format(memtableOffHeapSize, humanReadable);
- }
- statsTable.memtableSwitchCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableSwitchCount");
- statsTable.localReadCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
-
- double localReadLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getMean() / 1000;
- double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
- statsTable.localReadLatencyMs = localRLatency;
- statsTable.localWriteCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getCount();
-
- double localWriteLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getMean() / 1000;
- double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
- statsTable.localWriteLatencyMs = localWLatency;
- statsTable.pendingFlushes = probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
-
- statsTable.bloomFilterFalsePositives = probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterFalsePositives");
- statsTable.bloomFilterFalseRatio = probe.getColumnFamilyMetric(keyspaceName, tableName, "RecentBloomFilterFalseRatio");
- statsTable.bloomFilterSpaceUsed = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterDiskSpaceUsed"), humanReadable);
-
- if (bloomFilterOffHeapSize != null)
- {
- statsTable.bloomFilterOffHeapUsed = true;
- statsTable.bloomFilterOffHeapMemoryUsed = format(bloomFilterOffHeapSize, humanReadable);
- }
-
- if (indexSummaryOffHeapSize != null)
- {
- statsTable.indexSummaryOffHeapUsed = true;
- statsTable.indexSummaryOffHeapMemoryUsed = format(indexSummaryOffHeapSize, humanReadable);
- }
- if (compressionMetadataOffHeapSize != null)
- {
- statsTable.compressionMetadataOffHeapUsed = true;
- statsTable.compressionMetadataOffHeapMemoryUsed = format(compressionMetadataOffHeapSize, humanReadable);
- }
- statsTable.compactedPartitionMinimumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MinPartitionSize");
- statsTable.compactedPartitionMaximumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxPartitionSize");
- statsTable.compactedPartitionMeanBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MeanPartitionSize");
-
- CassandraMetricsRegistry.JmxHistogramMBean histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveScannedHistogram");
- statsTable.averageLiveCellsPerSliceLastFiveMinutes = histogram.getMean();
- statsTable.maximumLiveCellsPerSliceLastFiveMinutes = histogram.getMax();
-
- histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "TombstoneScannedHistogram");
- statsTable.averageTombstonesPerSliceLastFiveMinutes = histogram.getMean();
- statsTable.maximumTombstonesPerSliceLastFiveMinutes = histogram.getMax();
- statsTable.droppedMutations = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "DroppedMutations"), humanReadable);
- statsKeyspace.tables.add(statsTable);
- }
- holder.keyspaces.add(statsKeyspace);
- }
+ StatsHolder holder = new TableStatsHolder(probe, humanReadable, ignore, tableNames);
// print out the keyspace and table statistics
- TableStatsPrinter printer = TableStatsPrinter.from(outputFormat);
+ StatsPrinter printer = TableStatsPrinter.from(outputFormat);
printer.print(holder, System.out);
}
- private String format(long bytes, boolean humanReadable)
- {
- return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes);
- }
-
- /**
- * Used for filtering keyspaces and tables to be displayed using the tablestats command.
- */
- private static class OptionFilter
- {
- private final Map<String, List<String>> filter = new HashMap<>();
- private final Map<String, List<String>> verifier = new HashMap<>(); // Same as filter initially, but we remove tables every time we've checked them for inclusion
- // in isTableIncluded() so that we detect if those table requested don't exist (verifyTables())
- private final List<String> filterList = new ArrayList<>();
- private final boolean ignoreMode;
-
- OptionFilter(boolean ignoreMode, List<String> filterList)
- {
- this.filterList.addAll(filterList);
- this.ignoreMode = ignoreMode;
-
- for (String s : filterList)
- {
- String[] keyValues = s.split("\\.", 2);
-
- // build the map that stores the keyspaces and tables to use
- if (!filter.containsKey(keyValues[0]))
- {
- filter.put(keyValues[0], new ArrayList<>());
- verifier.put(keyValues[0], new ArrayList<>());
- }
-
- if (keyValues.length == 2)
- {
- filter.get(keyValues[0]).add(keyValues[1]);
- verifier.get(keyValues[0]).add(keyValues[1]);
- }
- }
- }
-
- public boolean isTableIncluded(String keyspace, String table)
- {
- // supplying empty params list is treated as wanting to display all keyspaces and tables
- if (filterList.isEmpty())
- return !ignoreMode;
-
- List<String> tables = filter.get(keyspace);
-
- // no such keyspace is in the map
- if (tables == null)
- return ignoreMode;
- // only a keyspace with no tables was supplied
- // so ignore or include (based on the flag) every column family in specified keyspace
- else if (tables.isEmpty())
- return !ignoreMode;
-
- // keyspace exists, and it contains specific table
- verifier.get(keyspace).remove(table);
- return ignoreMode ^ tables.contains(table);
- }
-
- public boolean isKeyspaceIncluded(String keyspace)
- {
- // supplying empty params list is treated as wanting to display all keyspaces and tables
- if (filterList.isEmpty())
- return !ignoreMode;
-
- // Note that if there is any table for the keyspace, we want to include the keyspace irregarding
- // of the ignoreMode, since the ignoreMode then apply to the table inside the keyspace but the
- // keyspace itself is not ignored
- return filter.get(keyspace) != null || ignoreMode;
- }
-
- public void verifyKeyspaces(List<String> keyspaces)
- {
- for (String ks : verifier.keySet())
- if (!keyspaces.contains(ks))
- throw new IllegalArgumentException("Unknown keyspace: " + ks);
- }
-
- public void verifyTables()
- {
- for (String ks : filter.keySet())
- if (!verifier.get(ks).isEmpty())
- throw new IllegalArgumentException("Unknown tables: " + verifier.get(ks) + " in keyspace: " + ks);
- }
- }
}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TpStats.java b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
index 5d3eab7..0cf8e50 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
@@ -19,36 +19,32 @@
import io.airlift.command.Command;
-import java.util.Map;
-
-import com.google.common.collect.Multimap;
-
+import io.airlift.command.Option;
import org.apache.cassandra.tools.NodeProbe;
import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+import org.apache.cassandra.tools.nodetool.stats.TpStatsHolder;
+import org.apache.cassandra.tools.nodetool.stats.TpStatsPrinter;
+import org.apache.cassandra.tools.nodetool.stats.*;
+
@Command(name = "tpstats", description = "Print usage statistics of thread pools")
public class TpStats extends NodeToolCmd
{
+ @Option(title = "format",
+ name = {"-F", "--format"},
+ description = "Output format (json, yaml)")
+ private String outputFormat = "";
+
@Override
public void execute(NodeProbe probe)
{
- System.out.printf("%-25s%10s%10s%15s%10s%18s%n", "Pool Name", "Active", "Pending", "Completed", "Blocked", "All time blocked");
-
-
- Multimap<String, String> threadPools = probe.getThreadPools();
- for (Map.Entry<String, String> tpool : threadPools.entries())
+ if (!outputFormat.isEmpty() && !"json".equals(outputFormat) && !"yaml".equals(outputFormat))
{
- System.out.printf("%-25s%10s%10s%15s%10s%18s%n",
- tpool.getValue(),
- probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "ActiveTasks"),
- probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "PendingTasks"),
- probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "CompletedTasks"),
- probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "CurrentlyBlockedTasks"),
- probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "TotalBlockedTasks"));
+ throw new IllegalArgumentException("arguments for -F are json,yaml only.");
}
- System.out.printf("%n%-20s%10s%n", "Message type", "Dropped");
- for (Map.Entry<String, Integer> entry : probe.getDroppedMessages().entrySet())
- System.out.printf("%-20s%10s%n", entry.getKey(), entry.getValue());
+ StatsHolder data = new TpStatsHolder(probe);
+ StatsPrinter printer = TpStatsPrinter.from(outputFormat);
+ printer.print(data, System.out);
}
}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
index e26f3f7..c35e1fe 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsHolder.java
@@ -18,88 +18,12 @@
package org.apache.cassandra.tools.nodetool.stats;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
import java.util.Map;
-public class StatsHolder
+/**
+ * Interface for the Stats property bag
+ */
+public interface StatsHolder
{
- public List<StatsKeyspace> keyspaces;
-
- public StatsHolder()
- {
- keyspaces = new ArrayList<>();
- }
-
- public Map<String, HashMap<String, Object>> convert2Map()
- {
- HashMap<String, HashMap<String, Object>> mpRet = new HashMap<>();
- for (StatsKeyspace keyspace : keyspaces)
- {
- // store each keyspace's metrics to map
- HashMap<String, Object> mpKeyspace = new HashMap<>();
- mpKeyspace.put("read_latency", keyspace.readLatency());
- mpKeyspace.put("read_count", keyspace.readCount);
- mpKeyspace.put("read_latency_ms", keyspace.readLatency());
- mpKeyspace.put("write_count", keyspace.writeCount);
- mpKeyspace.put("write_latency_ms", keyspace.writeLatency());
- mpKeyspace.put("pending_flushes", keyspace.pendingFlushes);
-
- // store each table's metrics to map
- List<StatsTable> tables = keyspace.tables;
- Map<String, Map<String, Object>> mpTables = new HashMap<>();
- for (StatsTable table : tables)
- {
- Map<String, Object> mpTable = new HashMap<>();
-
- mpTable.put("sstables_in_each_level", table.sstablesInEachLevel);
- mpTable.put("space_used_live", table.spaceUsedLive);
- mpTable.put("space_used_total", table.spaceUsedTotal);
- mpTable.put("space_used_by_snapshots_total", table.spaceUsedBySnapshotsTotal);
- if (table.offHeapUsed)
- mpTable.put("off_heap_memory_used_total", table.offHeapMemoryUsedTotal);
- mpTable.put("sstable_compression_ratio", table.sstableCompressionRatio);
- mpTable.put("number_of_keys_estimate", table.numberOfKeysEstimate);
- mpTable.put("memtable_cell_count", table.memtableCellCount);
- mpTable.put("memtable_data_size", table.memtableDataSize);
- if (table.memtableOffHeapUsed)
- mpTable.put("memtable_off_heap_memory_used", table.memtableOffHeapMemoryUsed);
- mpTable.put("memtable_switch_count", table.memtableSwitchCount);
- mpTable.put("local_read_count", table.localReadCount);
- mpTable.put("local_read_latency_ms", String.format("%01.3f", table.localReadLatencyMs));
- mpTable.put("local_write_count", table.localWriteCount);
- mpTable.put("local_write_latency_ms", String.format("%01.3f", table.localWriteLatencyMs));
- mpTable.put("pending_flushes", table.pendingFlushes);
- mpTable.put("percent_repaired", table.percentRepaired);
- mpTable.put("bloom_filter_false_positives", table.bloomFilterFalsePositives);
- mpTable.put("bloom_filter_false_ratio", String.format("%01.5f", table.bloomFilterFalseRatio));
- mpTable.put("bloom_filter_space_used", table.bloomFilterSpaceUsed);
- if (table.bloomFilterOffHeapUsed)
- mpTable.put("bloom_filter_off_heap_memory_used", table.bloomFilterOffHeapMemoryUsed);
- if (table.indexSummaryOffHeapUsed)
- mpTable.put("index_summary_off_heap_memory_used", table.indexSummaryOffHeapMemoryUsed);
- if (table.compressionMetadataOffHeapUsed)
- mpTable.put("compression_metadata_off_heap_memory_used",
- table.compressionMetadataOffHeapMemoryUsed);
- mpTable.put("compacted_partition_minimum_bytes", table.compactedPartitionMinimumBytes);
- mpTable.put("compacted_partition_maximum_bytes", table.compactedPartitionMaximumBytes);
- mpTable.put("compacted_partition_mean_bytes", table.compactedPartitionMeanBytes);
- mpTable.put("average_live_cells_per_slice_last_five_minutes",
- table.averageLiveCellsPerSliceLastFiveMinutes);
- mpTable.put("maximum_live_cells_per_slice_last_five_minutes",
- table.maximumLiveCellsPerSliceLastFiveMinutes);
- mpTable.put("average_tombstones_per_slice_last_five_minutes",
- table.averageTombstonesPerSliceLastFiveMinutes);
- mpTable.put("maximum_tombstones_per_slice_last_five_minutes",
- table.maximumTombstonesPerSliceLastFiveMinutes);
- mpTable.put("dropped_mutations", table.droppedMutations);
-
- mpTables.put(table.name, mpTable);
- }
- mpKeyspace.put("tables", mpTables);
- mpRet.put(keyspace.name, mpKeyspace);
- }
- return mpRet;
- }
-}
+ public Map<String, Object> convert2Map();
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
index 2d98781..e67f33a 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/StatsPrinter.java
@@ -15,11 +15,46 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.cassandra.tools.nodetool.stats;
import java.io.PrintStream;
-public interface StatsPrinter<T>
+import org.json.simple.JSONObject;
+import org.yaml.snakeyaml.DumperOptions;
+import org.yaml.snakeyaml.Yaml;
+
+/**
+ * Interface for the Stats printer, that'd output statistics
+ * given the {@code StatsHolder}
+ *
+ * @param <T> Stats property bad type
+ */
+public interface StatsPrinter<T extends StatsHolder>
{
- void printFormat(T data, PrintStream out);
-}
+ void print(T data, PrintStream out);
+
+ static class JsonPrinter<T extends StatsHolder> implements StatsPrinter<T>
+ {
+ @Override
+ public void print(T data, PrintStream out)
+ {
+ JSONObject json = new JSONObject();
+ json.putAll(data.convert2Map());
+ out.println(json.toString());
+ }
+ }
+
+ static class YamlPrinter<T extends StatsHolder> implements StatsPrinter<T>
+ {
+ @Override
+ public void print(T data, PrintStream out)
+ {
+ DumperOptions options = new DumperOptions();
+ options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK);
+
+ Yaml yaml = new Yaml(options);
+ out.println(yaml.dump(data.convert2Map()));
+ }
+ }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java
new file mode 100644
index 0000000..a785528
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java
@@ -0,0 +1,371 @@
+/*
+ * 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.nodetool.stats;
+
+import java.util.*;
+
+import javax.management.InstanceNotFoundException;
+
+import com.google.common.collect.ArrayListMultimap;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.metrics.*;
+import org.apache.cassandra.tools.*;
+
+public class TableStatsHolder implements StatsHolder
+{
+ public final List<StatsKeyspace> keyspaces;
+ public final int numberOfTables;
+
+ public TableStatsHolder(NodeProbe probe, boolean humanReadable, boolean ignore, List<String> tableNames)
+ {
+ this.keyspaces = new ArrayList<>();
+ this.numberOfTables = probe.getNumberOfTables();
+ this.initializeKeyspaces(probe, humanReadable, ignore, tableNames);
+ }
+
+ @Override
+ public Map<String, Object> convert2Map()
+ {
+ HashMap<String, Object> mpRet = new HashMap<>();
+ mpRet.put("total_number_of_tables", numberOfTables);
+ for (StatsKeyspace keyspace : keyspaces)
+ {
+ // store each keyspace's metrics to map
+ HashMap<String, Object> mpKeyspace = new HashMap<>();
+ mpKeyspace.put("read_latency", keyspace.readLatency());
+ mpKeyspace.put("read_count", keyspace.readCount);
+ mpKeyspace.put("read_latency_ms", keyspace.readLatency());
+ mpKeyspace.put("write_count", keyspace.writeCount);
+ mpKeyspace.put("write_latency_ms", keyspace.writeLatency());
+ mpKeyspace.put("pending_flushes", keyspace.pendingFlushes);
+
+ // store each table's metrics to map
+ List<StatsTable> tables = keyspace.tables;
+ Map<String, Map<String, Object>> mpTables = new HashMap<>();
+ for (StatsTable table : tables)
+ {
+ Map<String, Object> mpTable = new HashMap<>();
+
+ mpTable.put("sstables_in_each_level", table.sstablesInEachLevel);
+ mpTable.put("space_used_live", table.spaceUsedLive);
+ mpTable.put("space_used_total", table.spaceUsedTotal);
+ mpTable.put("space_used_by_snapshots_total", table.spaceUsedBySnapshotsTotal);
+ if (table.offHeapUsed)
+ mpTable.put("off_heap_memory_used_total", table.offHeapMemoryUsedTotal);
+ mpTable.put("sstable_compression_ratio", table.sstableCompressionRatio);
+ mpTable.put("number_of_keys_estimate", table.numberOfKeysEstimate);
+ mpTable.put("memtable_cell_count", table.memtableCellCount);
+ mpTable.put("memtable_data_size", table.memtableDataSize);
+ if (table.memtableOffHeapUsed)
+ mpTable.put("memtable_off_heap_memory_used", table.memtableOffHeapMemoryUsed);
+ mpTable.put("memtable_switch_count", table.memtableSwitchCount);
+ mpTable.put("local_read_count", table.localReadCount);
+ mpTable.put("local_read_latency_ms", String.format("%01.3f", table.localReadLatencyMs));
+ mpTable.put("local_write_count", table.localWriteCount);
+ mpTable.put("local_write_latency_ms", String.format("%01.3f", table.localWriteLatencyMs));
+ mpTable.put("pending_flushes", table.pendingFlushes);
+ mpTable.put("percent_repaired", table.percentRepaired);
+ mpTable.put("bloom_filter_false_positives", table.bloomFilterFalsePositives);
+ mpTable.put("bloom_filter_false_ratio", String.format("%01.5f", table.bloomFilterFalseRatio));
+ mpTable.put("bloom_filter_space_used", table.bloomFilterSpaceUsed);
+ if (table.bloomFilterOffHeapUsed)
+ mpTable.put("bloom_filter_off_heap_memory_used", table.bloomFilterOffHeapMemoryUsed);
+ if (table.indexSummaryOffHeapUsed)
+ mpTable.put("index_summary_off_heap_memory_used", table.indexSummaryOffHeapMemoryUsed);
+ if (table.compressionMetadataOffHeapUsed)
+ mpTable.put("compression_metadata_off_heap_memory_used",
+ table.compressionMetadataOffHeapMemoryUsed);
+ mpTable.put("compacted_partition_minimum_bytes", table.compactedPartitionMinimumBytes);
+ mpTable.put("compacted_partition_maximum_bytes", table.compactedPartitionMaximumBytes);
+ mpTable.put("compacted_partition_mean_bytes", table.compactedPartitionMeanBytes);
+ mpTable.put("average_live_cells_per_slice_last_five_minutes",
+ table.averageLiveCellsPerSliceLastFiveMinutes);
+ mpTable.put("maximum_live_cells_per_slice_last_five_minutes",
+ table.maximumLiveCellsPerSliceLastFiveMinutes);
+ mpTable.put("average_tombstones_per_slice_last_five_minutes",
+ table.averageTombstonesPerSliceLastFiveMinutes);
+ mpTable.put("maximum_tombstones_per_slice_last_five_minutes",
+ table.maximumTombstonesPerSliceLastFiveMinutes);
+ mpTable.put("dropped_mutations", table.droppedMutations);
+
+ mpTables.put(table.name, mpTable);
+ }
+ mpKeyspace.put("tables", mpTables);
+ mpRet.put(keyspace.name, mpKeyspace);
+ }
+ return mpRet;
+ }
+
+ private void initializeKeyspaces(NodeProbe probe, boolean humanReadable, boolean ignore, List<String> tableNames)
+ {
+ OptionFilter filter = new OptionFilter(ignore, tableNames);
+ ArrayListMultimap<String, ColumnFamilyStoreMBean> selectedTableMbeans = ArrayListMultimap.create();
+ Map<String, StatsKeyspace> keyspaceStats = new HashMap<>();
+
+ // get a list of table stores
+ Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tableMBeans = probe.getColumnFamilyStoreMBeanProxies();
+
+ while (tableMBeans.hasNext())
+ {
+ Map.Entry<String, ColumnFamilyStoreMBean> entry = tableMBeans.next();
+ String keyspaceName = entry.getKey();
+ ColumnFamilyStoreMBean tableProxy = entry.getValue();
+
+ if (filter.isKeyspaceIncluded(keyspaceName))
+ {
+ StatsKeyspace stats = keyspaceStats.get(keyspaceName);
+ if (stats == null)
+ {
+ stats = new StatsKeyspace(probe, keyspaceName);
+ keyspaceStats.put(keyspaceName, stats);
+ }
+ stats.add(tableProxy);
+
+ if (filter.isTableIncluded(keyspaceName, tableProxy.getTableName()))
+ selectedTableMbeans.put(keyspaceName, tableProxy);
+ }
+ }
+
+ // make sure all specified keyspace and tables exist
+ filter.verifyKeyspaces(probe.getKeyspaces());
+ filter.verifyTables();
+
+ // get metrics of keyspace
+ for (Map.Entry<String, Collection<ColumnFamilyStoreMBean>> entry : selectedTableMbeans.asMap().entrySet())
+ {
+ String keyspaceName = entry.getKey();
+ Collection<ColumnFamilyStoreMBean> tables = entry.getValue();
+ StatsKeyspace statsKeyspace = keyspaceStats.get(keyspaceName);
+
+ // get metrics of table statistics for this keyspace
+ for (ColumnFamilyStoreMBean table : tables)
+ {
+ String tableName = table.getTableName();
+ StatsTable statsTable = new StatsTable();
+ statsTable.name = tableName;
+ statsTable.isIndex = tableName.contains(".");
+ statsTable.sstableCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveSSTableCount");
+ int[] leveledSStables = table.getSSTableCountPerLevel();
+ if (leveledSStables != null)
+ {
+ statsTable.isLeveledSstable = true;
+
+ for (int level = 0; level < leveledSStables.length; level++)
+ {
+ int count = leveledSStables[level];
+ long maxCount = 4L; // for L0
+ if (level > 0)
+ maxCount = (long) Math.pow(10, level);
+ // show max threshold for level when exceeded
+ statsTable.sstablesInEachLevel.add(count + ((count > maxCount) ? "/" + maxCount : ""));
+ }
+ }
+
+ Long memtableOffHeapSize = null;
+ Long bloomFilterOffHeapSize = null;
+ Long indexSummaryOffHeapSize = null;
+ Long compressionMetadataOffHeapSize = null;
+ Long offHeapSize = null;
+ Double percentRepaired = null;
+
+ try
+ {
+ memtableOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableOffHeapSize");
+ bloomFilterOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterOffHeapMemoryUsed");
+ indexSummaryOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "IndexSummaryOffHeapMemoryUsed");
+ compressionMetadataOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionMetadataOffHeapMemoryUsed");
+ offHeapSize = memtableOffHeapSize + bloomFilterOffHeapSize + indexSummaryOffHeapSize + compressionMetadataOffHeapSize;
+ percentRepaired = (Double) probe.getColumnFamilyMetric(keyspaceName, tableName, "PercentRepaired");
+ }
+ catch (RuntimeException e)
+ {
+ // offheap-metrics introduced in 2.1.3 - older versions do not have the appropriate mbeans
+ if (!(e.getCause() instanceof InstanceNotFoundException))
+ throw e;
+ }
+
+ statsTable.spaceUsedLive = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), humanReadable);
+ statsTable.spaceUsedTotal = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), humanReadable);
+ statsTable.spaceUsedBySnapshotsTotal = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), humanReadable);
+ if (offHeapSize != null)
+ {
+ statsTable.offHeapUsed = true;
+ statsTable.offHeapMemoryUsedTotal = format(offHeapSize, humanReadable);
+
+ }
+ if (percentRepaired != null)
+ {
+ statsTable.percentRepaired = Math.round(100 * percentRepaired) / 100.0;
+ }
+ statsTable.sstableCompressionRatio = probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionRatio");
+ Object estimatedPartitionCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "EstimatedPartitionCount");
+ if (Long.valueOf(-1L).equals(estimatedPartitionCount))
+ {
+ estimatedPartitionCount = 0L;
+ }
+ statsTable.numberOfKeysEstimate = estimatedPartitionCount;
+
+ statsTable.memtableCellCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableColumnsCount");
+ statsTable.memtableDataSize = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), humanReadable);
+ if (memtableOffHeapSize != null)
+ {
+ statsTable.memtableOffHeapUsed = true;
+ statsTable.memtableOffHeapMemoryUsed = format(memtableOffHeapSize, humanReadable);
+ }
+ statsTable.memtableSwitchCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableSwitchCount");
+ statsTable.localReadCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
+
+ double localReadLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getMean() / 1000;
+ double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
+ statsTable.localReadLatencyMs = localRLatency;
+ statsTable.localWriteCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getCount();
+
+ double localWriteLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getMean() / 1000;
+ double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
+ statsTable.localWriteLatencyMs = localWLatency;
+ statsTable.pendingFlushes = probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
+
+ statsTable.bloomFilterFalsePositives = probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterFalsePositives");
+ statsTable.bloomFilterFalseRatio = probe.getColumnFamilyMetric(keyspaceName, tableName, "RecentBloomFilterFalseRatio");
+ statsTable.bloomFilterSpaceUsed = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterDiskSpaceUsed"), humanReadable);
+
+ if (bloomFilterOffHeapSize != null)
+ {
+ statsTable.bloomFilterOffHeapUsed = true;
+ statsTable.bloomFilterOffHeapMemoryUsed = format(bloomFilterOffHeapSize, humanReadable);
+ }
+
+ if (indexSummaryOffHeapSize != null)
+ {
+ statsTable.indexSummaryOffHeapUsed = true;
+ statsTable.indexSummaryOffHeapMemoryUsed = format(indexSummaryOffHeapSize, humanReadable);
+ }
+ if (compressionMetadataOffHeapSize != null)
+ {
+ statsTable.compressionMetadataOffHeapUsed = true;
+ statsTable.compressionMetadataOffHeapMemoryUsed = format(compressionMetadataOffHeapSize, humanReadable);
+ }
+ statsTable.compactedPartitionMinimumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MinPartitionSize");
+ statsTable.compactedPartitionMaximumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxPartitionSize");
+ statsTable.compactedPartitionMeanBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MeanPartitionSize");
+
+ CassandraMetricsRegistry.JmxHistogramMBean histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveScannedHistogram");
+ statsTable.averageLiveCellsPerSliceLastFiveMinutes = histogram.getMean();
+ statsTable.maximumLiveCellsPerSliceLastFiveMinutes = histogram.getMax();
+
+ histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "TombstoneScannedHistogram");
+ statsTable.averageTombstonesPerSliceLastFiveMinutes = histogram.getMean();
+ statsTable.maximumTombstonesPerSliceLastFiveMinutes = histogram.getMax();
+ statsTable.droppedMutations = format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "DroppedMutations"), humanReadable);
+ statsKeyspace.tables.add(statsTable);
+ }
+ keyspaces.add(statsKeyspace);
+ }
+ }
+
+ private String format(long bytes, boolean humanReadable)
+ {
+ return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes);
+ }
+
+ /**
+ * Used for filtering keyspaces and tables to be displayed using the tablestats command.
+ */
+ private static class OptionFilter
+ {
+ private final Map<String, List<String>> filter = new HashMap<>();
+ private final Map<String, List<String>> verifier = new HashMap<>(); // Same as filter initially, but we remove tables every time we've checked them for inclusion
+ // in isTableIncluded() so that we detect if those table requested don't exist (verifyTables())
+ private final List<String> filterList = new ArrayList<>();
+ private final boolean ignoreMode;
+
+ OptionFilter(boolean ignoreMode, List<String> filterList)
+ {
+ this.filterList.addAll(filterList);
+ this.ignoreMode = ignoreMode;
+
+ for (String s : filterList)
+ {
+ String[] keyValues = s.split("\\.", 2);
+
+ // build the map that stores the keyspaces and tables to use
+ if (!filter.containsKey(keyValues[0]))
+ {
+ filter.put(keyValues[0], new ArrayList<>());
+ verifier.put(keyValues[0], new ArrayList<>());
+ }
+
+ if (keyValues.length == 2)
+ {
+ filter.get(keyValues[0]).add(keyValues[1]);
+ verifier.get(keyValues[0]).add(keyValues[1]);
+ }
+ }
+ }
+
+ public boolean isTableIncluded(String keyspace, String table)
+ {
+ // supplying empty params list is treated as wanting to display all keyspaces and tables
+ if (filterList.isEmpty())
+ return !ignoreMode;
+
+ List<String> tables = filter.get(keyspace);
+
+ // no such keyspace is in the map
+ if (tables == null)
+ return ignoreMode;
+ // only a keyspace with no tables was supplied
+ // so ignore or include (based on the flag) every column family in specified keyspace
+ else if (tables.isEmpty())
+ return !ignoreMode;
+
+ // keyspace exists, and it contains specific table
+ verifier.get(keyspace).remove(table);
+ return ignoreMode ^ tables.contains(table);
+ }
+
+ public boolean isKeyspaceIncluded(String keyspace)
+ {
+ // supplying empty params list is treated as wanting to display all keyspaces and tables
+ if (filterList.isEmpty())
+ return !ignoreMode;
+
+ // Note that if there is any table for the keyspace, we want to include the keyspace irregarding
+ // of the ignoreMode, since the ignoreMode then apply to the table inside the keyspace but the
+ // keyspace itself is not ignored
+ return filter.get(keyspace) != null || ignoreMode;
+ }
+
+ public void verifyKeyspaces(List<String> keyspaces)
+ {
+ for (String ks : verifier.keySet())
+ if (!keyspaces.contains(ks))
+ throw new IllegalArgumentException("Unknown keyspace: " + ks);
+ }
+
+ public void verifyTables()
+ {
+ for (String ks : filter.keySet())
+ if (!verifier.get(ks).isEmpty())
+ throw new IllegalArgumentException("Unknown tables: " + verifier.get(ks) + " in keyspace: " + ks);
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java
index a6da189..49888ee 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsPrinter.java
@@ -15,50 +15,35 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.cassandra.tools.nodetool.stats;
import java.io.PrintStream;
import java.util.List;
-import org.json.simple.JSONObject;
-import org.yaml.snakeyaml.Yaml;
-
-public enum TableStatsPrinter
+public class TableStatsPrinter
{
- DEFAULT(new DefaultPrinter()),
- JSON(new JsonPrinter()),
- YAML(new YamlPrinter()),;
-
- private final StatsPrinter<StatsHolder> printer;
-
- TableStatsPrinter(StatsPrinter<StatsHolder> printer)
- {
- this.printer = printer;
- }
-
- public void print(StatsHolder stats, PrintStream out)
- {
- printer.printFormat(stats, out);
- }
-
- public static TableStatsPrinter from(String format)
+ public static StatsPrinter from(String format)
{
switch (format)
{
case "json":
- return JSON;
+ return new StatsPrinter.JsonPrinter();
case "yaml":
- return YAML;
+ return new StatsPrinter.YamlPrinter();
default:
- return DEFAULT;
+ return new DefaultPrinter();
}
}
- private static class DefaultPrinter implements StatsPrinter<StatsHolder>
+ private static class DefaultPrinter implements StatsPrinter<TableStatsHolder>
{
@Override
- public void printFormat(StatsHolder data, PrintStream out)
+ public void print(TableStatsHolder data, PrintStream out)
{
+ out.println("Total number of tables: " + data.numberOfTables);
+ out.println("----------------");
+
List<StatsKeyspace> keyspaces = data.keyspaces;
for (StatsKeyspace keyspace : keyspaces)
{
@@ -125,26 +110,4 @@
}
}
}
-
- private static class JsonPrinter implements StatsPrinter<StatsHolder>
- {
- @Override
- public void printFormat(StatsHolder data, PrintStream out)
- {
- JSONObject json = new JSONObject();
- json.putAll(data.convert2Map());
- out.println(json.toString());
- }
- }
-
- private static class YamlPrinter implements StatsPrinter<StatsHolder>
- {
- @Override
- public void printFormat(StatsHolder data, PrintStream out)
- {
- Yaml yaml = new Yaml();
- out.println(yaml.dump(data.convert2Map()));
- }
- }
-
}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsHolder.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsHolder.java
new file mode 100644
index 0000000..d70b4dd
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsHolder.java
@@ -0,0 +1,61 @@
+/*
+ * 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.nodetool.stats;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.nodetool.stats.StatsHolder;
+
+public class TpStatsHolder implements StatsHolder
+{
+ public final NodeProbe probe;
+
+ public TpStatsHolder(NodeProbe probe)
+ {
+ this.probe = probe;
+ }
+
+ @Override
+ public Map<String, Object> convert2Map()
+ {
+ HashMap<String, Object> result = new HashMap<>();
+ HashMap<String, Map<String, Object>> threadPools = new HashMap<>();
+ HashMap<String, Object> droppedMessage = new HashMap<>();
+
+ for (Map.Entry<String, String> tp : probe.getThreadPools().entries())
+ {
+ HashMap<String, Object> threadPool = new HashMap<>();
+ threadPool.put("ActiveTasks", probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "ActiveTasks"));
+ threadPool.put("PendingTasks", probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "PendingTasks"));
+ threadPool.put("CompletedTasks", probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "CompletedTasks"));
+ threadPool.put("CurrentlyBlockedTasks", probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "CurrentlyBlockedTasks"));
+ threadPool.put("TotalBlockedTasks", probe.getThreadPoolMetric(tp.getKey(), tp.getValue(), "TotalBlockedTasks"));
+ threadPools.put(tp.getValue(), threadPool);
+ }
+ result.put("ThreadPools", threadPools);
+
+ for (Map.Entry<String, Integer> entry : probe.getDroppedMessages().entrySet())
+ droppedMessage.put(entry.getKey(), entry.getValue());
+ result.put("DroppedMessage", droppedMessage);
+
+ return result;
+ }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsPrinter.java b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsPrinter.java
new file mode 100644
index 0000000..b874746
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/stats/TpStatsPrinter.java
@@ -0,0 +1,73 @@
+/*
+ * 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.nodetool.stats;
+
+import java.io.PrintStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TpStatsPrinter
+{
+ public static StatsPrinter from(String format)
+ {
+ switch (format)
+ {
+ case "json":
+ return new StatsPrinter.JsonPrinter();
+ case "yaml":
+ return new StatsPrinter.YamlPrinter();
+ default:
+ return new DefaultPrinter();
+ }
+
+ }
+
+ public static class DefaultPrinter implements StatsPrinter<TpStatsHolder>
+ {
+ @Override
+ public void print(TpStatsHolder data, PrintStream out)
+ {
+ Map<String, Object> convertData = data.convert2Map();
+
+ out.printf("%-30s%10s%10s%15s%10s%18s%n", "Pool Name", "Active", "Pending", "Completed", "Blocked", "All time blocked");
+
+ Map<Object, Object> threadPools = convertData.get("ThreadPools") instanceof Map<?, ?> ? (Map)convertData.get("ThreadPools") : Collections.emptyMap();
+ for (Map.Entry<Object, Object> entry : threadPools.entrySet())
+ {
+ Map values = entry.getValue() instanceof Map<?, ?> ? (Map)entry.getValue() : Collections.emptyMap();
+ out.printf("%-30s%10s%10s%15s%10s%18s%n",
+ entry.getKey(),
+ values.get("ActiveTasks"),
+ values.get("PendingTasks"),
+ values.get("CompletedTasks"),
+ values.get("CurrentlyBlockedTasks"),
+ values.get("TotalBlockedTasks"));
+ }
+
+ out.printf("%n%-20s%10s%n", "Message type", "Dropped");
+
+ Map<Object, Object> droppedMessages = convertData.get("DroppedMessage") instanceof Map<?, ?> ? (Map)convertData.get("DroppedMessage") : Collections.emptyMap();
+ for (Map.Entry<Object, Object> entry : droppedMessages.entrySet())
+ {
+ out.printf("%-20s%10s%n", entry.getKey(), entry.getValue());
+ }
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java b/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
index bc8d5dd..9230d38 100644
--- a/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
+++ b/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
@@ -47,4 +47,9 @@
{
delegate.waitForPendingEvents();
}
+
+ TraceState getDelegate()
+ {
+ return delegate;
+ }
}
diff --git a/src/java/org/apache/cassandra/tracing/TraceState.java b/src/java/org/apache/cassandra/tracing/TraceState.java
index ec2bc9e..b4eff6b 100644
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@ -181,7 +181,10 @@
protected abstract void traceImpl(String message);
- protected abstract void waitForPendingEvents();
+ protected void waitForPendingEvents()
+ {
+ // if tracing events are asynchronous, then you can use this method to wait for them to complete
+ }
public boolean acquireReference()
{
diff --git a/src/java/org/apache/cassandra/tracing/TraceStateImpl.java b/src/java/org/apache/cassandra/tracing/TraceStateImpl.java
index e2d3a68..55e8389 100644
--- a/src/java/org/apache/cassandra/tracing/TraceStateImpl.java
+++ b/src/java/org/apache/cassandra/tracing/TraceStateImpl.java
@@ -19,9 +19,13 @@
import java.net.InetAddress;
import java.util.Collections;
+import java.util.Set;
import java.util.UUID;
-import java.util.concurrent.ExecutionException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,6 +49,8 @@
private static final int WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS =
Integer.valueOf(System.getProperty("cassandra.wait_for_tracing_events_timeout_secs", "1"));
+ private final Set<Future<?>> pendingFutures = ConcurrentHashMap.newKeySet();
+
public TraceStateImpl(InetAddress coordinator, UUID sessionId, Tracing.TraceType traceType)
{
super(coordinator, sessionId, traceType);
@@ -61,9 +67,7 @@
}
/**
- * Post a no-op event to the TRACING stage, so that we can be sure that any previous mutations
- * have at least been applied to one replica. This works because the tracking executor only
- * has one thread in its pool, see {@link StageManager#tracingExecutor()}.
+ * Wait on submitted futures
*/
protected void waitForPendingEvents()
{
@@ -73,28 +77,39 @@
try
{
if (logger.isTraceEnabled())
- logger.trace("Waiting for up to {} seconds for trace events to complete",
- WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS);
+ logger.trace("Waiting for up to {} seconds for {} trace events to complete",
+ +WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS, pendingFutures.size());
- StageManager.getStage(Stage.TRACING).submit(StageManager.NO_OP_TASK)
- .get(WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS, TimeUnit.SECONDS);
+ CompletableFuture.allOf(pendingFutures.toArray(new CompletableFuture<?>[pendingFutures.size()]))
+ .get(WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS, TimeUnit.SECONDS);
+ }
+ catch (TimeoutException ex)
+ {
+ if (logger.isTraceEnabled())
+ logger.trace("Failed to wait for tracing events to complete in {} seconds",
+ WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS);
}
catch (Throwable t)
{
JVMStabilityInspector.inspectThrowable(t);
- logger.debug("Failed to wait for tracing events to complete: {}", t);
+ logger.error("Got exception whilst waiting for tracing events to complete", t);
}
}
- static void executeMutation(final Mutation mutation)
+
+ void executeMutation(final Mutation mutation)
{
- StageManager.getStage(Stage.TRACING).execute(new WrappedRunnable()
+ CompletableFuture<Void> fut = CompletableFuture.runAsync(new WrappedRunnable()
{
protected void runMayThrow()
{
mutateWithCatch(mutation);
}
- });
+ }, StageManager.getStage(Stage.TRACING));
+
+ boolean ret = pendingFutures.add(fut);
+ if (!ret)
+ logger.warn("Failed to insert pending future, tracing synchronization may not work");
}
static void mutateWithCatch(Mutation mutation)
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index e69645f..adf5ed9 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -31,6 +31,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import io.netty.util.concurrent.FastThreadLocal;
import org.apache.cassandra.concurrent.ExecutorLocal;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.marshal.TimeUUIDType;
@@ -84,7 +85,7 @@
private final InetAddress localAddress = FBUtilities.getLocalAddress();
- private final ThreadLocal<TraceState> state = new ThreadLocal<>();
+ private final FastThreadLocal<TraceState> state = new FastThreadLocal<>();
protected final ConcurrentMap<UUID, TraceState> sessions = new ConcurrentHashMap<>();
diff --git a/src/java/org/apache/cassandra/tracing/TracingImpl.java b/src/java/org/apache/cassandra/tracing/TracingImpl.java
index 52ac183..4f69584 100644
--- a/src/java/org/apache/cassandra/tracing/TracingImpl.java
+++ b/src/java/org/apache/cassandra/tracing/TracingImpl.java
@@ -28,10 +28,6 @@
import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.utils.WrappedRunnable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
/**
* A trace session context. Able to track and store trace sessions. A session is usually a user initiated query, and may
@@ -39,31 +35,62 @@
*/
class TracingImpl extends Tracing
{
- private static final Logger logger = LoggerFactory.getLogger(TracingImpl.class);
-
public void stopSessionImpl() {
- TraceState state = get();
+ final TraceStateImpl state = getStateImpl();
+ if (state == null)
+ return;
+
int elapsed = state.elapsed();
ByteBuffer sessionId = state.sessionIdBytes;
int ttl = state.ttl;
- TraceStateImpl.executeMutation(TraceKeyspace.makeStopSessionMutation(sessionId, elapsed, ttl));
+
+ state.executeMutation(TraceKeyspace.makeStopSessionMutation(sessionId, elapsed, ttl));
}
public TraceState begin(final String request, final InetAddress client, final Map<String, String> parameters)
{
assert isTracing();
- final TraceState state = get();
+ final TraceStateImpl state = getStateImpl();
+ assert state != null;
+
final long startedAt = System.currentTimeMillis();
final ByteBuffer sessionId = state.sessionIdBytes;
final String command = state.traceType.toString();
final int ttl = state.ttl;
- TraceStateImpl.executeMutation(TraceKeyspace.makeStartSessionMutation(sessionId, client, parameters, request, startedAt, command, ttl));
-
+ state.executeMutation(TraceKeyspace.makeStartSessionMutation(sessionId, client, parameters, request, startedAt, command, ttl));
return state;
}
+ /**
+ * Convert the abstract tracing state to its implementation.
+ *
+ * Expired states are not put in the sessions but the check is for extra safety.
+ *
+ * @return the state converted to its implementation, or null
+ */
+ private TraceStateImpl getStateImpl()
+ {
+ TraceState state = get();
+ if (state == null)
+ return null;
+
+ if (state instanceof ExpiredTraceState)
+ {
+ ExpiredTraceState expiredTraceState = (ExpiredTraceState) state;
+ state = expiredTraceState.getDelegate();
+ }
+
+ if (state instanceof TraceStateImpl)
+ {
+ return (TraceStateImpl)state;
+ }
+
+ assert false : "TracingImpl states should be of type TraceStateImpl";
+ return null;
+ }
+
@Override
protected TraceState newTraceState(InetAddress coordinator, UUID sessionId, TraceType traceType)
{
diff --git a/src/java/org/apache/cassandra/transport/CBUtil.java b/src/java/org/apache/cassandra/transport/CBUtil.java
index 43f4bbd..570fd6b 100644
--- a/src/java/org/apache/cassandra/transport/CBUtil.java
+++ b/src/java/org/apache/cassandra/transport/CBUtil.java
@@ -393,7 +393,7 @@
int length = cb.readInt();
if (length < 0)
{
- if (protocolVersion < 4) // backward compatibility for pre-version 4
+ if (protocolVersion < Server.VERSION_4) // backward compatibility for pre-version 4
return null;
if (length == -1)
return null;
diff --git a/src/java/org/apache/cassandra/transport/Event.java b/src/java/org/apache/cassandra/transport/Event.java
index 3c45c33..232d817 100644
--- a/src/java/org/apache/cassandra/transport/Event.java
+++ b/src/java/org/apache/cassandra/transport/Event.java
@@ -270,7 +270,7 @@
public static SchemaChange deserializeEvent(ByteBuf cb, int version)
{
Change change = CBUtil.readEnumValue(Change.class, cb);
- if (version >= 3)
+ if (version >= Server.VERSION_3)
{
Target target = CBUtil.readEnumValue(Target.class, cb);
String keyspace = CBUtil.readString(cb);
@@ -293,7 +293,7 @@
{
if (target == Target.FUNCTION || target == Target.AGGREGATE)
{
- if (version >= 4)
+ if (version >= Server.VERSION_4)
{
// available since protocol version 4
CBUtil.writeEnumValue(change, dest);
@@ -314,7 +314,7 @@
return;
}
- if (version >= 3)
+ if (version >= Server.VERSION_3)
{
CBUtil.writeEnumValue(change, dest);
CBUtil.writeEnumValue(target, dest);
@@ -345,13 +345,13 @@
{
if (target == Target.FUNCTION || target == Target.AGGREGATE)
{
- if (version >= 4)
+ if (version >= Server.VERSION_3)
return CBUtil.sizeOfEnumValue(change)
+ CBUtil.sizeOfEnumValue(target)
+ CBUtil.sizeOfString(keyspace)
+ CBUtil.sizeOfString(name)
+ CBUtil.sizeOfStringList(argTypes);
- if (version >= 3)
+ if (version >= Server.VERSION_3)
return CBUtil.sizeOfEnumValue(Change.UPDATED)
+ CBUtil.sizeOfEnumValue(Target.KEYSPACE)
+ CBUtil.sizeOfString(keyspace);
@@ -360,7 +360,7 @@
+ CBUtil.sizeOfString("");
}
- if (version >= 3)
+ if (version >= Server.VERSION_3)
{
int size = CBUtil.sizeOfEnumValue(change)
+ CBUtil.sizeOfEnumValue(target)
diff --git a/src/java/org/apache/cassandra/transport/Frame.java b/src/java/org/apache/cassandra/transport/Frame.java
index d0d4aee..f2f6174 100644
--- a/src/java/org/apache/cassandra/transport/Frame.java
+++ b/src/java/org/apache/cassandra/transport/Frame.java
@@ -85,11 +85,6 @@
public final int streamId;
public final Message.Type type;
- private Header(int version, int flags, int streamId, Message.Type type)
- {
- this(version, Flag.deserialize(flags), streamId, type);
- }
-
private Header(int version, EnumSet<Flag> flags, int streamId, Message.Type type)
{
this.version = version;
@@ -104,7 +99,8 @@
COMPRESSED,
TRACING,
CUSTOM_PAYLOAD,
- WARNING;
+ WARNING,
+ USE_BETA;
private static final Flag[] ALL_VALUES = values();
@@ -174,16 +170,26 @@
int firstByte = buffer.getByte(idx++);
Message.Direction direction = Message.Direction.extractFromVersion(firstByte);
int version = firstByte & PROTOCOL_VERSION_MASK;
- if (version < Server.MIN_SUPPORTED_VERSION || version > Server.CURRENT_VERSION)
- throw new ProtocolException(String.format("Invalid or unsupported protocol version (%d); the lowest supported version is %d and the greatest is %d",
- version, Server.MIN_SUPPORTED_VERSION, Server.CURRENT_VERSION),
- version);
+ if (version < Server.MIN_SUPPORTED_VERSION)
+ throw new ProtocolException(protocolVersionExceptionMessage(version), version);
// Wait until we have the complete header
if (readableBytes < Header.LENGTH)
return;
int flags = buffer.getByte(idx++);
+ EnumSet<Header.Flag> decodedFlags = Header.Flag.deserialize(flags);
+ if (version > Server.CURRENT_VERSION)
+ {
+ if (version == Server.BETA_VERSION)
+ {
+ if (!decodedFlags.contains(Header.Flag.USE_BETA))
+ throw new ProtocolException(String.format("Beta version of the protocol used (%d), but USE_BETA flag is unset",
+ version));
+ }
+ else
+ throw new ProtocolException(protocolVersionExceptionMessage(version));
+ }
int streamId = buffer.getShort(idx);
idx += 2;
@@ -242,7 +248,13 @@
streamId);
}
- results.add(new Frame(new Header(version, flags, streamId, type), body));
+ results.add(new Frame(new Header(version, decodedFlags, streamId, type), body));
+ }
+
+ private static String protocolVersionExceptionMessage(int version)
+ {
+ return String.format("Invalid or unsupported protocol version (%d); the lowest supported version is %d and the greatest is %d",
+ version, Server.MIN_SUPPORTED_VERSION, Server.CURRENT_VERSION);
}
private void fail()
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index 7bfa194..66e0014 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -322,6 +322,8 @@
int version = connection == null ? Server.CURRENT_VERSION : connection.getVersion();
EnumSet<Frame.Header.Flag> flags = EnumSet.noneOf(Frame.Header.Flag.class);
+ if (version == Server.BETA_VERSION)
+ flags.add(Frame.Header.Flag.USE_BETA);
Codec<Message> codec = (Codec<Message>)message.type.codec;
try
diff --git a/src/java/org/apache/cassandra/transport/ProtocolException.java b/src/java/org/apache/cassandra/transport/ProtocolException.java
index a589e9b..9d8c270 100644
--- a/src/java/org/apache/cassandra/transport/ProtocolException.java
+++ b/src/java/org/apache/cassandra/transport/ProtocolException.java
@@ -35,6 +35,8 @@
public ProtocolException(String msg, Integer attemptedLowProtocolVersion)
{
super(msg);
+ assert attemptedLowProtocolVersion == null || attemptedLowProtocolVersion < Server.MIN_SUPPORTED_VERSION;
+
this.attemptedLowProtocolVersion = attemptedLowProtocolVersion;
}
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index 388fca0..36ad186 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -66,7 +66,9 @@
public static final int VERSION_3 = 3;
public static final int VERSION_4 = 4;
+ public static final int VERSION_5 = 5;
public static final int CURRENT_VERSION = VERSION_4;
+ public static final int BETA_VERSION = VERSION_5;
public static final int MIN_SUPPORTED_VERSION = VERSION_3;
private final ConnectionTracker connectionTracker = new ConnectionTracker();
diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java
index 6e20cfa..4d8a30b 100644
--- a/src/java/org/apache/cassandra/transport/SimpleClient.java
+++ b/src/java/org/apache/cassandra/transport/SimpleClient.java
@@ -92,10 +92,7 @@
public SimpleClient(String host, int port, int version, ClientEncryptionOptions encryptionOptions)
{
- this.host = host;
- this.port = port;
- this.version = version;
- this.encryptionOptions = encryptionOptions;
+ this(host, port, version, false, encryptionOptions);
}
public SimpleClient(String host, int port, ClientEncryptionOptions encryptionOptions)
@@ -108,6 +105,17 @@
this(host, port, version, new ClientEncryptionOptions());
}
+ public SimpleClient(String host, int port, int version, boolean useBeta, ClientEncryptionOptions encryptionOptions)
+ {
+ this.host = host;
+ this.port = port;
+ if (version == Server.BETA_VERSION && !useBeta)
+ throw new IllegalArgumentException(String.format("Beta version of server used (%s), but USE_BETA flag is not set", version));
+
+ this.version = version;
+ this.encryptionOptions = encryptionOptions;
+ }
+
public SimpleClient(String host, int port)
{
this(host, port, new ClientEncryptionOptions());
diff --git a/src/java/org/apache/cassandra/transport/messages/AuthResponse.java b/src/java/org/apache/cassandra/transport/messages/AuthResponse.java
index ca7a0c3..8b3e866 100644
--- a/src/java/org/apache/cassandra/transport/messages/AuthResponse.java
+++ b/src/java/org/apache/cassandra/transport/messages/AuthResponse.java
@@ -23,6 +23,7 @@
import org.apache.cassandra.auth.AuthenticatedUser;
import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.exceptions.AuthenticationException;
+import org.apache.cassandra.metrics.AuthMetrics;
import org.apache.cassandra.service.QueryState;
import org.apache.cassandra.transport.*;
@@ -77,6 +78,7 @@
{
AuthenticatedUser user = negotiator.getAuthenticatedUser();
queryState.getClientState().login(user);
+ AuthMetrics.instance.markSuccess();
// authentication is complete, send a ready message to the client
return new AuthSuccess(challenge);
}
@@ -87,6 +89,7 @@
}
catch (AuthenticationException e)
{
+ AuthMetrics.instance.markFailure();
return ErrorMessage.fromException(e);
}
}
diff --git a/src/java/org/apache/cassandra/transport/messages/BatchMessage.java b/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
index bd2423e..9d1047f 100644
--- a/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
@@ -84,7 +84,7 @@
CBUtil.writeValueList(msg.values.get(i), dest);
}
- if (version < 3)
+ if (version < Server.VERSION_3)
CBUtil.writeConsistencyLevel(msg.options.getConsistency(), dest);
else
QueryOptions.codec.encode(msg.options, dest, version);
@@ -102,7 +102,7 @@
size += CBUtil.sizeOfValueList(msg.values.get(i));
}
- size += version < 3
+ size += version < Server.VERSION_3
? CBUtil.sizeOfConsistencyLevel(msg.options.getConsistency())
: QueryOptions.codec.encodedSize(msg.options, version);
return size;
diff --git a/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java b/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java
index fc959ab..4c51cce 100644
--- a/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java
@@ -24,6 +24,7 @@
import org.apache.cassandra.auth.AuthenticatedUser;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.exceptions.AuthenticationException;
+import org.apache.cassandra.metrics.AuthMetrics;
import org.apache.cassandra.service.QueryState;
import org.apache.cassandra.transport.CBUtil;
import org.apache.cassandra.transport.Message;
@@ -76,9 +77,11 @@
{
AuthenticatedUser user = DatabaseDescriptor.getAuthenticator().legacyAuthenticate(credentials);
state.getClientState().login(user);
+ AuthMetrics.instance.markSuccess();
}
catch (AuthenticationException e)
{
+ AuthMetrics.instance.markFailure();
return ErrorMessage.fromException(e);
}
diff --git a/src/java/org/apache/cassandra/triggers/ITrigger.java b/src/java/org/apache/cassandra/triggers/ITrigger.java
index ad631d1..d46c036 100644
--- a/src/java/org/apache/cassandra/triggers/ITrigger.java
+++ b/src/java/org/apache/cassandra/triggers/ITrigger.java
@@ -21,7 +21,6 @@
*/
-import java.nio.ByteBuffer;
import java.util.Collection;
import org.apache.cassandra.db.Mutation;
diff --git a/src/java/org/apache/cassandra/utils/Architecture.java b/src/java/org/apache/cassandra/utils/Architecture.java
new file mode 100644
index 0000000..3173f8e
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/Architecture.java
@@ -0,0 +1,44 @@
+/*
+* 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.Collections;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
+
+public final class Architecture
+{
+ // Note that s390x architecture are not officially supported and adding it here is only done out of convenience
+ // for those that want to run C* on this architecture at their own risk (see #11214)
+ private static final Set<String> UNALIGNED_ARCH = Collections.unmodifiableSet(Sets.newHashSet(
+ "i386",
+ "x86",
+ "amd64",
+ "x86_64",
+ "s390x"
+ ));
+
+ public static final boolean IS_UNALIGNED = UNALIGNED_ARCH.contains(System.getProperty("os.arch"));
+
+ private Architecture()
+ {
+ }
+}
diff --git a/src/java/org/apache/cassandra/utils/BackgroundActivityMonitor.java b/src/java/org/apache/cassandra/utils/BackgroundActivityMonitor.java
deleted file mode 100644
index 1799d10..0000000
--- a/src/java/org/apache/cassandra/utils/BackgroundActivityMonitor.java
+++ /dev/null
@@ -1,171 +0,0 @@
-package org.apache.cassandra.utils;
-/*
- *
- * 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.io.IOException;
-import java.io.RandomAccessFile;
-import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
-import java.util.StringTokenizer;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
-import org.apache.cassandra.gms.ApplicationState;
-import org.apache.cassandra.gms.EndpointState;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.service.StorageService;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.util.concurrent.AtomicDouble;
-
-public class BackgroundActivityMonitor
-{
- private static final Logger logger = LoggerFactory.getLogger(BackgroundActivityMonitor.class);
-
- public static final int USER_INDEX = 0;
- public static final int NICE_INDEX = 1;
- public static final int SYS_INDEX = 2;
- public static final int IDLE_INDEX = 3;
- public static final int IOWAIT_INDEX = 4;
- public static final int IRQ_INDEX = 5;
- public static final int SOFTIRQ_INDEX = 6;
-
- private static final int NUM_CPUS = Runtime.getRuntime().availableProcessors();
- private static final String PROC_STAT_PATH = "/proc/stat";
-
- private final AtomicDouble compaction_severity = new AtomicDouble();
- private final AtomicDouble manual_severity = new AtomicDouble();
- private final ScheduledExecutorService reportThread = new DebuggableScheduledThreadPoolExecutor("Background_Reporter");
-
- private RandomAccessFile statsFile;
- private long[] lastReading;
-
- public BackgroundActivityMonitor()
- {
- try
- {
- statsFile = new RandomAccessFile(PROC_STAT_PATH, "r");
- lastReading = readAndCompute();
- }
- catch (IOException ex)
- {
- if (FBUtilities.hasProcFS())
- logger.warn("Couldn't open /proc/stats");
- statsFile = null;
- }
- reportThread.scheduleAtFixedRate(new BackgroundActivityReporter(), 1, 1, TimeUnit.SECONDS);
- }
-
- private long[] readAndCompute() throws IOException
- {
- statsFile.seek(0);
- StringTokenizer tokenizer = new StringTokenizer(statsFile.readLine());
- String name = tokenizer.nextToken();
- assert name.equalsIgnoreCase("cpu");
- long[] returned = new long[tokenizer.countTokens()];
- for (int i = 0; i < returned.length; i++)
- returned[i] = Long.parseLong(tokenizer.nextToken());
- return returned;
- }
-
- private float compareAtIndex(long[] reading1, long[] reading2, int index)
- {
- long total1 = 0, total2 = 0;
- for (int i = 0; i <= SOFTIRQ_INDEX; i++)
- {
- total1 += reading1[i];
- total2 += reading2[i];
- }
- float totalDiff = total2 - total1;
-
- long intrested1 = reading1[index], intrested2 = reading2[index];
- float diff = intrested2 - intrested1;
- if (diff == 0)
- return 0f;
- return (diff / totalDiff) * 100; // yes it is hard coded to 100 [update
- // unit?]
- }
-
- public void incrCompactionSeverity(double sev)
- {
- compaction_severity.addAndGet(sev);
- }
-
- public void incrManualSeverity(double sev)
- {
- manual_severity.addAndGet(sev);
- }
-
- public double getIOWait() throws IOException
- {
- if (statsFile == null)
- return -1d;
- long[] newComp = readAndCompute();
- double value = compareAtIndex(lastReading, newComp, IOWAIT_INDEX);
- lastReading = newComp;
- return value;
- }
-
- public double getNormalizedLoadAvg()
- {
- double avg = ManagementFactory.getOperatingSystemMXBean().getSystemLoadAverage();
- return avg / NUM_CPUS;
- }
-
- public double getSeverity(InetAddress endpoint)
- {
- VersionedValue event;
- EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
- if (state != null && (event = state.getApplicationState(ApplicationState.SEVERITY)) != null)
- return Double.parseDouble(event.value);
- return 0.0;
- }
-
- public class BackgroundActivityReporter implements Runnable
- {
- public void run()
- {
- double report = -1;
- try
- {
- report = getIOWait();
- }
- catch (IOException e)
- {
- // ignore;
- if (FBUtilities.hasProcFS())
- logger.warn("Couldn't read /proc/stats");
- }
- if (report == -1d)
- report = compaction_severity.get();
-
- if (!Gossiper.instance.isEnabled())
- return;
- report += manual_severity.get(); // add manual severity setting.
- VersionedValue updated = StorageService.instance.valueFactory.severity(report);
- Gossiper.instance.addLocalApplicationState(ApplicationState.SEVERITY, updated);
- }
- }
-}
diff --git a/src/java/org/apache/cassandra/utils/CLibrary.java b/src/java/org/apache/cassandra/utils/CLibrary.java
index 347b0df..a68db95 100644
--- a/src/java/org/apache/cassandra/utils/CLibrary.java
+++ b/src/java/org/apache/cassandra/utils/CLibrary.java
@@ -23,8 +23,6 @@
import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.channels.FileChannel;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
import java.util.concurrent.TimeUnit;
import org.slf4j.Logger;
diff --git a/src/java/org/apache/cassandra/utils/ChecksumType.java b/src/java/org/apache/cassandra/utils/ChecksumType.java
index 3fa245b..413a171 100644
--- a/src/java/org/apache/cassandra/utils/ChecksumType.java
+++ b/src/java/org/apache/cassandra/utils/ChecksumType.java
@@ -22,6 +22,8 @@
import java.util.zip.CRC32;
import java.util.zip.Adler32;
+import io.netty.util.concurrent.FastThreadLocal;
+
public enum ChecksumType
{
Adler32
@@ -60,12 +62,13 @@
public abstract Checksum newInstance();
public abstract void update(Checksum checksum, ByteBuffer buf);
- private ThreadLocal<Checksum> instances = ThreadLocal.withInitial(this::newInstance);
-
- public Checksum threadLocalInstance()
+ private FastThreadLocal<Checksum> instances = new FastThreadLocal<Checksum>()
{
- return instances.get();
- }
+ protected Checksum initialValue() throws Exception
+ {
+ return newInstance();
+ }
+ };
public long of(ByteBuffer buf)
{
diff --git a/src/java/org/apache/cassandra/utils/Clock.java b/src/java/org/apache/cassandra/utils/Clock.java
new file mode 100644
index 0000000..eb9822c
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/Clock.java
@@ -0,0 +1,80 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Wrapper around time related functions that are either implemented by using the default JVM calls
+ * or by using a custom implementation for testing purposes.
+ *
+ * See {@link #instance} for how to use a custom implementation.
+ *
+ * Please note that {@link java.time.Clock} wasn't used, as it would not be possible to provide an
+ * implementation for {@link #nanoTime()} with the exact same properties of {@link System#nanoTime()}.
+ */
+public class Clock
+{
+ private static final Logger logger = LoggerFactory.getLogger(Clock.class);
+
+ /**
+ * Static singleton object that will be instanciated by default with a system clock
+ * implementation. Set <code>cassandra.clock</code> system property to a FQCN to use a
+ * different implementation instead.
+ */
+ public static Clock instance;
+
+ static
+ {
+ String sclock = System.getProperty("cassandra.clock");
+ if (sclock == null)
+ {
+ instance = new Clock();
+ }
+ else
+ {
+ try
+ {
+ logger.debug("Using custom clock implementation: {}", sclock);
+ instance = (Clock) Class.forName(sclock).newInstance();
+ }
+ catch (Exception e)
+ {
+ logger.error(e.getMessage(), e);
+ }
+ }
+ }
+
+ /**
+ * @see System#nanoTime()
+ */
+ public long nanoTime()
+ {
+ return System.nanoTime();
+ }
+
+ /**
+ * @see System#currentTimeMillis()
+ */
+ public long currentTimeMillis()
+ {
+ return System.currentTimeMillis();
+ }
+
+}
diff --git a/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java b/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java
index aa7898c..c1fb6e0 100644
--- a/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java
+++ b/src/java/org/apache/cassandra/utils/DirectorySizeCalculator.java
@@ -24,39 +24,19 @@
import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor;
import java.nio.file.attribute.BasicFileAttributes;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.google.common.collect.ImmutableSet;
-
-import static com.google.common.collect.Sets.newHashSet;
/**
* Walks directory recursively, summing up total contents of files within.
*/
public class DirectorySizeCalculator extends SimpleFileVisitor<Path>
{
- protected final AtomicLong size = new AtomicLong(0);
- protected Set<String> visited = newHashSet(); //count each file only once
- protected Set<String> alive = newHashSet();
+ protected volatile long size = 0;
protected final File path;
public DirectorySizeCalculator(File path)
{
super();
this.path = path;
- rebuildFileList();
- }
-
- public DirectorySizeCalculator(List<File> files)
- {
- super();
- this.path = null;
- ImmutableSet.Builder<String> builder = ImmutableSet.builder();
- for (File file : files)
- builder.add(file.getName());
- alive = builder.build();
}
public boolean isAcceptable(Path file)
@@ -64,24 +44,11 @@
return true;
}
- public void rebuildFileList()
- {
- assert path != null;
- ImmutableSet.Builder<String> builder = ImmutableSet.builder();
- for (File file : path.listFiles())
- builder.add(file.getName());
- size.set(0);
- alive = builder.build();
- }
-
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException
{
if (isAcceptable(file))
- {
- size.addAndGet(attrs.size());
- visited.add(file.toFile().getName());
- }
+ size += attrs.size();
return FileVisitResult.CONTINUE;
}
@@ -93,6 +60,6 @@
public long getAllocatedSize()
{
- return size.get();
+ return size;
}
}
diff --git a/src/java/org/apache/cassandra/utils/ExpiringMap.java b/src/java/org/apache/cassandra/utils/ExpiringMap.java
index e7b626c..fc88880 100644
--- a/src/java/org/apache/cassandra/utils/ExpiringMap.java
+++ b/src/java/org/apache/cassandra/utils/ExpiringMap.java
@@ -48,7 +48,7 @@
assert value != null;
this.value = value;
this.timeout = timeout;
- this.createdAt = System.nanoTime();
+ this.createdAt = Clock.instance.nanoTime();
}
private boolean isReadyToDieAt(long atNano)
@@ -85,7 +85,7 @@
{
public void run()
{
- long start = System.nanoTime();
+ long start = Clock.instance.nanoTime();
int n = 0;
for (Map.Entry<K, CacheableObject<V>> entry : cache.entrySet())
{
diff --git a/src/java/org/apache/cassandra/utils/FastByteOperations.java b/src/java/org/apache/cassandra/utils/FastByteOperations.java
index cf8d305..02c0dbb 100644
--- a/src/java/org/apache/cassandra/utils/FastByteOperations.java
+++ b/src/java/org/apache/cassandra/utils/FastByteOperations.java
@@ -102,10 +102,7 @@
*/
static ByteOperations getBest()
{
- String arch = System.getProperty("os.arch");
- boolean unaligned = arch.equals("i386") || arch.equals("x86")
- || arch.equals("amd64") || arch.equals("x86_64") || arch.equals("s390x");
- if (!unaligned)
+ if (!Architecture.IS_UNALIGNED)
return new PureJavaOperations();
try
{
diff --git a/src/java/org/apache/cassandra/utils/Interval.java b/src/java/org/apache/cassandra/utils/Interval.java
index 335ef27..9398144 100644
--- a/src/java/org/apache/cassandra/utils/Interval.java
+++ b/src/java/org/apache/cassandra/utils/Interval.java
@@ -17,8 +17,6 @@
*/
package org.apache.cassandra.utils;
-import java.util.Comparator;
-
import com.google.common.base.Objects;
public class Interval<C, D>
diff --git a/src/java/org/apache/cassandra/utils/MD5Digest.java b/src/java/org/apache/cassandra/utils/MD5Digest.java
index 2feb09e..4e736dc 100644
--- a/src/java/org/apache/cassandra/utils/MD5Digest.java
+++ b/src/java/org/apache/cassandra/utils/MD5Digest.java
@@ -17,6 +17,7 @@
*/
package org.apache.cassandra.utils;
+import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
@@ -55,6 +56,11 @@
return compute(toHash.getBytes(StandardCharsets.UTF_8));
}
+ public ByteBuffer byteBuffer()
+ {
+ return ByteBuffer.wrap(bytes);
+ }
+
@Override
public final int hashCode()
{
diff --git a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java b/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java
index 69b7a47..5aafbe5 100644
--- a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java
+++ b/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java
@@ -22,8 +22,6 @@
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.config.Config;
-import com.google.common.annotations.VisibleForTesting;
-
/*
* Convert from nanotime to non-monotonic current time millis. Beware of weaker ordering guarantees.
*/
diff --git a/src/java/org/apache/cassandra/utils/ObjectSizes.java b/src/java/org/apache/cassandra/utils/ObjectSizes.java
index e7469c1..5c2434e 100644
--- a/src/java/org/apache/cassandra/utils/ObjectSizes.java
+++ b/src/java/org/apache/cassandra/utils/ObjectSizes.java
@@ -23,8 +23,6 @@
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-
import org.github.jamm.MemoryLayoutSpecification;
import org.github.jamm.MemoryMeter;
diff --git a/src/java/org/apache/cassandra/utils/StatusLogger.java b/src/java/org/apache/cassandra/utils/StatusLogger.java
index 712e8f7..c33190b 100644
--- a/src/java/org/apache/cassandra/utils/StatusLogger.java
+++ b/src/java/org/apache/cassandra/utils/StatusLogger.java
@@ -19,19 +19,14 @@
import java.lang.management.ManagementFactory;
import java.util.Map;
-import java.util.Set;
import javax.management.*;
-import com.google.common.collect.Iterables;
-
import org.apache.cassandra.cache.*;
-import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.metrics.ThreadPoolMetrics;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.RowIndexEntry;
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index a8b3093..a72139d 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -29,7 +29,6 @@
import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Charsets;
import com.google.common.primitives.Ints;
/**
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/src/java/org/apache/cassandra/utils/Wrapped.java
similarity index 61%
copy from src/java/org/apache/cassandra/io/util/ICompressedFile.java
copy to src/java/org/apache/cassandra/utils/Wrapped.java
index e69487c..1996a86 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/src/java/org/apache/cassandra/utils/Wrapped.java
@@ -15,12 +15,34 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.compress.CompressionMetadata;
+package org.apache.cassandra.utils;
-public interface ICompressedFile
+/**
+ * Simple wrapper class to be used when a lambda function
+ * needs to modify a variable outside it's scope.
+ */
+public class Wrapped<T>
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ private T value;
+
+ public static <V> Wrapped<V> create(V initial)
+ {
+ return new Wrapped<>(initial);
+ }
+
+ private Wrapped(T initial)
+ {
+ this.value = initial;
+ }
+
+ public T get()
+ {
+ return value;
+ }
+
+ public void set(T value)
+ {
+ this.value = value;
+ }
}
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/src/java/org/apache/cassandra/utils/WrappedBoolean.java
similarity index 68%
copy from src/java/org/apache/cassandra/io/util/ICompressedFile.java
copy to src/java/org/apache/cassandra/utils/WrappedBoolean.java
index e69487c..4b1443e 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/src/java/org/apache/cassandra/utils/WrappedBoolean.java
@@ -15,12 +15,28 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.compress.CompressionMetadata;
+package org.apache.cassandra.utils;
-public interface ICompressedFile
+/**
+ * Simple wrapper for native boolean type
+ */
+public class WrappedBoolean
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ private boolean value;
+
+ public WrappedBoolean(boolean initial)
+ {
+ this.value = initial;
+ }
+
+ public boolean get()
+ {
+ return value;
+ }
+
+ public void set(boolean value)
+ {
+ this.value = value;
+ }
}
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/src/java/org/apache/cassandra/utils/WrappedException.java
similarity index 77%
rename from src/java/org/apache/cassandra/io/util/ICompressedFile.java
rename to src/java/org/apache/cassandra/utils/WrappedException.java
index e69487c..3cf56bc 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/src/java/org/apache/cassandra/utils/WrappedException.java
@@ -15,12 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.compress.CompressionMetadata;
+package org.apache.cassandra.utils;
-public interface ICompressedFile
+/**
+ * Wrapped runtime exception for lambda functions
+ */
+public class WrappedException extends RuntimeException
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ public WrappedException(Exception cause)
+ {
+ super(cause);
+ }
}
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/src/java/org/apache/cassandra/utils/WrappedInt.java
similarity index 63%
copy from src/java/org/apache/cassandra/io/util/ICompressedFile.java
copy to src/java/org/apache/cassandra/utils/WrappedInt.java
index e69487c..a106575 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/src/java/org/apache/cassandra/utils/WrappedInt.java
@@ -15,12 +15,38 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.compress.CompressionMetadata;
+package org.apache.cassandra.utils;
-public interface ICompressedFile
+/**
+ * Simple wrapper for native int type
+ */
+public class WrappedInt
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ private int value;
+
+ public WrappedInt(int initial)
+ {
+ this.value = initial;
+ }
+
+ public int get()
+ {
+ return value;
+ }
+
+ public void set(int value)
+ {
+ this.value = value;
+ }
+
+ public void increment()
+ {
+ ++value;
+ }
+
+ public void decrement()
+ {
+ --value;
+ }
}
diff --git a/src/java/org/apache/cassandra/utils/btree/BTree.java b/src/java/org/apache/cassandra/utils/btree/BTree.java
index 33f4152..5a6fffe 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTree.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTree.java
@@ -19,8 +19,10 @@
package org.apache.cassandra.utils.btree;
import java.util.*;
+import java.util.function.Consumer;
import com.google.common.base.Function;
+import com.google.common.base.Predicate;
import com.google.common.collect.Iterators;
import com.google.common.collect.Ordering;
@@ -136,7 +138,8 @@
for (K k : source)
values[i++] = updateF.apply(k);
}
- updateF.allocated(ObjectSizes.sizeOfArray(values));
+ if (updateF != UpdateFunction.noOp())
+ updateF.allocated(ObjectSizes.sizeOfArray(values));
return values;
}
@@ -188,7 +191,7 @@
tree1 = tree2;
tree2 = tmp;
}
- return update(tree1, comparator, new BTreeSet<K>(tree2, comparator), updateF);
+ return update(tree1, comparator, new BTreeSet<>(tree2, comparator), updateF);
}
public static <V> Iterator<V> iterator(Object[] btree)
@@ -1159,4 +1162,128 @@
}
return compare(cmp, previous, max) < 0;
}
+
+ /**
+ * Simple method to walk the btree forwards or reversed and apply a function to each element
+ *
+ * Public method
+ *
+ */
+ public static <V> void apply(Object[] btree, Consumer<V> function, boolean reversed)
+ {
+ if (reversed)
+ applyReverse(btree, function, null);
+ else
+ applyForwards(btree, function, null);
+ }
+
+ /**
+ * Simple method to walk the btree forwards or reversed and apply a function till a stop condition is reached
+ *
+ * Public method
+ *
+ */
+ public static <V> void apply(Object[] btree, Consumer<V> function, Predicate<V> stopCondition, boolean reversed)
+ {
+ if (reversed)
+ applyReverse(btree, function, stopCondition);
+ else
+ applyForwards(btree, function, stopCondition);
+ }
+
+
+
+
+ /**
+ * Simple method to walk the btree forwards and apply a function till a stop condition is reached
+ *
+ * Private method
+ *
+ * @param btree
+ * @param function
+ * @param stopCondition
+ */
+ private static <V> boolean applyForwards(Object[] btree, Consumer<V> function, Predicate<V> stopCondition)
+ {
+ boolean isLeaf = isLeaf(btree);
+ int childOffset = isLeaf ? Integer.MAX_VALUE : getChildStart(btree);
+ int limit = isLeaf ? getLeafKeyEnd(btree) : btree.length - 1;
+ for (int i = 0 ; i < limit ; i++)
+ {
+ // we want to visit in iteration order, so we visit our key nodes inbetween our children
+ int idx = isLeaf ? i : (i / 2) + (i % 2 == 0 ? childOffset : 0);
+ Object current = btree[idx];
+ if (idx < childOffset)
+ {
+ V castedCurrent = (V) current;
+ if (stopCondition != null && stopCondition.apply(castedCurrent))
+ return true;
+
+ function.accept(castedCurrent);
+ }
+ else
+ {
+ if (applyForwards((Object[]) current, function, stopCondition))
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ /**
+ * Simple method to walk the btree in reverse and apply a function till a stop condition is reached
+ *
+ * Private method
+ *
+ * @param btree
+ * @param function
+ * @param stopCondition
+ */
+ private static <V> boolean applyReverse(Object[] btree, Consumer<V> function, Predicate<V> stopCondition)
+ {
+ boolean isLeaf = isLeaf(btree);
+ int childOffset = isLeaf ? 0 : getChildStart(btree);
+ int limit = isLeaf ? getLeafKeyEnd(btree) : btree.length - 1;
+ for (int i = limit - 1, visited = 0; i >= 0 ; i--, visited++)
+ {
+ int idx = i;
+
+ // we want to visit in reverse iteration order, so we visit our children nodes inbetween our keys
+ if (!isLeaf)
+ {
+ int typeOffset = visited / 2;
+
+ if (i % 2 == 0)
+ {
+ // This is a child branch. Since children are in the second half of the array, we must
+ // adjust for the key's we've visited along the way
+ idx += typeOffset;
+ }
+ else
+ {
+ // This is a key. Since the keys are in the first half of the array and we are iterating
+ // in reverse we subtract the childOffset and adjust for children we've walked so far
+ idx = i - childOffset + typeOffset;
+ }
+ }
+
+ Object current = btree[idx];
+ if (isLeaf || idx < childOffset)
+ {
+ V castedCurrent = (V) current;
+ if (stopCondition != null && stopCondition.apply(castedCurrent))
+ return true;
+
+ function.accept(castedCurrent);
+ }
+ else
+ {
+ if (applyReverse((Object[]) current, function, stopCondition))
+ return true;
+ }
+ }
+
+ return false;
+ }
}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/RefCounted.java b/src/java/org/apache/cassandra/utils/concurrent/RefCounted.java
index e68c7bd..d09d288 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/RefCounted.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/RefCounted.java
@@ -18,23 +18,6 @@
*/
package org.apache.cassandra.utils.concurrent;
-import java.lang.ref.PhantomReference;
-import java.lang.ref.ReferenceQueue;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterators;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.NamedThreadFactory;
-
/**
* An object that needs ref counting does the two following:
* - defines a Tidy object that will cleanup once it's gone,
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Refs.java b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
index ec8d6c0..e5d9c37 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Refs.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
@@ -25,7 +25,6 @@
import javax.annotation.Nullable;
import com.google.common.base.Function;
-import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapPool.java b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
index 593b443..46f4111 100644
--- a/src/java/org/apache/cassandra/utils/memory/HeapPool.java
+++ b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
@@ -18,13 +18,6 @@
*/
package org.apache.cassandra.utils.memory;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.utils.concurrent.OpOrder;
-
public class HeapPool extends MemtablePool
{
public HeapPool(long maxOnHeapMemory, float cleanupThreshold, Runnable cleaner)
diff --git a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
index 3a18964..6c2e6fd 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
@@ -23,6 +23,9 @@
import java.nio.ByteOrder;
import com.sun.jna.Native;
+
+import org.apache.cassandra.utils.Architecture;
+
import sun.misc.Unsafe;
import sun.nio.ch.DirectBuffer;
@@ -44,17 +47,10 @@
private static final boolean BIG_ENDIAN = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
- private static final boolean UNALIGNED;
- public static final boolean INVERTED_ORDER;
+ public static final boolean INVERTED_ORDER = Architecture.IS_UNALIGNED && !BIG_ENDIAN;
static
{
- String arch = System.getProperty("os.arch");
- // Note that s390x architecture are not officially supported and adding it here is only done out of convenience
- // for those that want to run C* on this architecture at their own risk (see #11214)
- UNALIGNED = arch.equals("i386") || arch.equals("x86")
- || arch.equals("amd64") || arch.equals("x86_64") || arch.equals("s390x");
- INVERTED_ORDER = UNALIGNED && !BIG_ENDIAN;
try
{
Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
@@ -120,7 +116,7 @@
public static void setInt(long address, int l)
{
- if (UNALIGNED)
+ if (Architecture.IS_UNALIGNED)
unsafe.putInt(address, l);
else
putIntByByte(address, l);
@@ -128,7 +124,7 @@
public static void setLong(long address, long l)
{
- if (UNALIGNED)
+ if (Architecture.IS_UNALIGNED)
unsafe.putLong(address, l);
else
putLongByByte(address, l);
@@ -141,17 +137,17 @@
public static int getShort(long address)
{
- return (UNALIGNED ? unsafe.getShort(address) : getShortByByte(address)) & 0xffff;
+ return (Architecture.IS_UNALIGNED ? unsafe.getShort(address) : getShortByByte(address)) & 0xffff;
}
public static int getInt(long address)
{
- return UNALIGNED ? unsafe.getInt(address) : getIntByByte(address);
+ return Architecture.IS_UNALIGNED ? unsafe.getInt(address) : getIntByByte(address);
}
public static long getLong(long address)
{
- return UNALIGNED ? unsafe.getLong(address) : getLongByByte(address);
+ return Architecture.IS_UNALIGNED ? unsafe.getLong(address) : getLongByByte(address);
}
public static ByteBuffer getByteBuffer(long address, int length)
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java b/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
index fb35b38..fe412ea 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
@@ -19,7 +19,6 @@
import java.nio.ByteBuffer;
-import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.utils.concurrent.OpOrder;
diff --git a/src/java/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupport.java b/src/java/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupport.java
index 275673e..a33fe7d 100644
--- a/src/java/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupport.java
+++ b/src/java/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupport.java
@@ -102,6 +102,6 @@
protected static int getCmd(String tag)
{
- return Integer.valueOf(tag.split(":")[1]);
+ return Integer.parseInt(tag.split(":")[1]);
}
}
diff --git a/src/resources/org/apache/cassandra/cql3/functions/JavaSourceUDF.txt b/src/resources/org/apache/cassandra/cql3/functions/JavaSourceUDF.txt
index d736a5a..802081f 100644
--- a/src/resources/org/apache/cassandra/cql3/functions/JavaSourceUDF.txt
+++ b/src/resources/org/apache/cassandra/cql3/functions/JavaSourceUDF.txt
@@ -25,6 +25,14 @@
return super.decompose(protocolVersion, result);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ #return_type# result = #execute_internal_name#(
+#arguments_aggregate#
+ );
+ return result;
+ }
+
private #return_type# #execute_internal_name#(#argument_list#)
{
#body#
diff --git a/test/conf/logback-test.xml b/test/conf/logback-test.xml
index f6a5492..94cff40 100644
--- a/test/conf/logback-test.xml
+++ b/test/conf/logback-test.xml
@@ -45,7 +45,7 @@
<appender name="STDOUT" target="System.out" class="org.apache.cassandra.ConsoleAppender">
<encoder>
- <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
+ <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
</encoder>
<filter class="ch.qos.logback.classic.filter.ThresholdFilter">
<level>DEBUG</level>
diff --git a/test/long/org/apache/cassandra/cql3/ViewLongTest.java b/test/long/org/apache/cassandra/cql3/ViewLongTest.java
index 3808b73..4dded85 100644
--- a/test/long/org/apache/cassandra/cql3/ViewLongTest.java
+++ b/test/long/org/apache/cassandra/cql3/ViewLongTest.java
@@ -32,9 +32,6 @@
import com.datastax.driver.core.Row;
import com.datastax.driver.core.exceptions.NoHostAvailableException;
import com.datastax.driver.core.exceptions.WriteTimeoutException;
-import org.apache.cassandra.concurrent.SEPExecutor;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.batchlog.BatchlogManager;
import org.apache.cassandra.utils.WrappedRunnable;
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index 239077e..babd571 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -29,9 +29,16 @@
import java.util.concurrent.atomic.AtomicLong;
import com.google.common.util.concurrent.RateLimiter;
-import org.junit.*;
-import org.apache.cassandra.*;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import io.netty.util.concurrent.FastThreadLocalThread;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.UpdateBuilder;
import org.apache.cassandra.config.Config.CommitLogSync;
import org.apache.cassandra.config.*;
import org.apache.cassandra.db.Mutation;
@@ -97,6 +104,7 @@
initialize();
CommitLogStressTest tester = new CommitLogStressTest();
+ tester.cleanDir();
tester.testFixedSize();
}
catch (Throwable e)
@@ -130,6 +138,7 @@
SchemaLoader.loadSchema();
SchemaLoader.schemaDefinition(""); // leave def. blank to maintain old behaviour
+
CommitLog.instance.stopUnsafe(true);
}
@@ -208,9 +217,7 @@
for (CommitLogSync sync : CommitLogSync.values())
{
DatabaseDescriptor.setCommitLogSync(sync);
- CommitLog commitLog = new CommitLog(CommitLogArchiver.disabled());
- commitLog.segmentManager.enableReserveSegmentCreation();
- commitLog.start();
+ CommitLog commitLog = new CommitLog(CommitLogArchiver.disabled()).start();
testLog(commitLog);
assert !failed;
}
@@ -307,17 +314,12 @@
private void verifySizes(CommitLog commitLog)
{
// Complete anything that's still left to write.
- commitLog.executor.requestExtraSync().awaitUninterruptibly();
- // One await() does not suffice as we may be signalled when an ongoing sync finished. Request another
- // (which shouldn't write anything) to make sure the first we triggered completes.
- // FIXME: The executor should give us a chance to await completion of the sync we requested.
- commitLog.executor.requestExtraSync().awaitUninterruptibly();
-
- // Wait for any pending deletes or segment allocations to complete.
+ commitLog.executor.syncBlocking();
+ // Wait for any concurrent segment allocations to complete.
commitLog.segmentManager.awaitManagementTasksCompletion();
long combinedSize = 0;
- for (File f : new File(DatabaseDescriptor.getCommitLogLocation()).listFiles())
+ for (File f : new File(commitLog.segmentManager.storageDirectory).listFiles())
combinedSize += f.length();
Assert.assertEquals(combinedSize, commitLog.getActiveOnDiskSize());
@@ -407,7 +409,8 @@
return slice;
}
- public class CommitlogThread extends Thread {
+ public class CommitlogThread extends FastThreadLocalThread
+ {
final AtomicLong counter = new AtomicLong();
int hash = 0;
int cells = 0;
diff --git a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
index 0d66fa9..2349453 100644
--- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
+++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
@@ -31,8 +31,6 @@
import org.apache.cassandra.utils.FBUtilities;
-import static org.junit.Assert.assertEquals;
-
public class DynamicEndpointSnitchLongTest
{
@Test
diff --git a/test/microbench/org/apache/cassandra/test/microbench/DirectorySizerBench.java b/test/microbench/org/apache/cassandra/test/microbench/DirectorySizerBench.java
index a653c81..34cbb17 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/DirectorySizerBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/DirectorySizerBench.java
@@ -99,7 +99,6 @@
@Benchmark
public void countFiles(final Blackhole bh) throws IOException
{
- sizer.rebuildFileList();
Files.walkFileTree(tempDir.toPath(), sizer);
}
}
diff --git a/test/microbench/org/apache/cassandra/test/microbench/FastThreadExecutor.java b/test/microbench/org/apache/cassandra/test/microbench/FastThreadExecutor.java
new file mode 100644
index 0000000..d0b4442
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/FastThreadExecutor.java
@@ -0,0 +1,96 @@
+/*
+ * 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.test.microbench;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import io.netty.util.concurrent.FastThreadLocalThread;
+
+/**
+ * Created to test perf of FastThreadLocal
+ *
+ * Used in MutationBench via:
+ * jvmArgsAppend = {"-Djmh.executor=CUSTOM", "-Djmh.executor.class=org.apache.cassandra.test.microbench.FastThreadExecutor"}
+ */
+public class FastThreadExecutor extends AbstractExecutorService
+{
+ final FastThreadLocalThread thread;
+ final LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<>();
+ final CountDownLatch shutdown = new CountDownLatch(1);
+
+ public FastThreadExecutor(int size, String name)
+ {
+ assert size == 1;
+
+ thread = new FastThreadLocalThread(() -> {
+ Runnable work = null;
+ try
+ {
+ while ((work = queue.take()) != null)
+ work.run();
+ }
+ catch (InterruptedException e)
+ {
+ shutdown.countDown();
+ }
+ });
+
+ thread.setName(name + "-1");
+ thread.setDaemon(true);
+
+ thread.start();
+ }
+
+
+ public void shutdown()
+ {
+ thread.interrupt();
+ }
+
+ public List<Runnable> shutdownNow()
+ {
+ thread.interrupt();
+ return Collections.emptyList();
+ }
+
+ public boolean isShutdown()
+ {
+ return shutdown.getCount() == 0;
+ }
+
+ public boolean isTerminated()
+ {
+ return shutdown.getCount() == 0;
+ }
+
+ public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException
+ {
+ return shutdown.await(timeout, unit);
+ }
+
+ public void execute(Runnable command)
+ {
+ while(!queue.add(command));
+ }
+}
diff --git a/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
new file mode 100644
index 0000000..89dcb0f
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
@@ -0,0 +1,148 @@
+/*
+ * 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.test.microbench;
+
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.concurrent.*;
+
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
+import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.profile.StackProfiler;
+import org.openjdk.jmh.results.Result;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+@BenchmarkMode(Mode.Throughput)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+@Warmup(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS)
+@Measurement(iterations = 5, time = 2, timeUnit = TimeUnit.SECONDS)
+@Fork(value = 1
+ , jvmArgsAppend = {"-Djmh.executor=CUSTOM", "-Djmh.executor.class=org.apache.cassandra.test.microbench.FastThreadExecutor"
+ //,"-XX:+UnlockCommercialFeatures", "-XX:+FlightRecorder","-XX:+UnlockDiagnosticVMOptions", "-XX:+DebugNonSafepoints",
+ // "-XX:StartFlightRecording=duration=60s,filename=./profiling-data.jfr,name=profile,settings=profile",
+ // "-XX:FlightRecorderOptions=settings=/home/jake/workspace/cassandra/profiling-advanced.jfc,samplethreads=true"
+ }
+)
+@Threads(1)
+@State(Scope.Benchmark)
+public class MutationBench
+{
+ static
+ {
+ Config.setClientMode(true);
+ // Partitioner is not set in client mode.
+ if (DatabaseDescriptor.getPartitioner() == null)
+ DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
+ }
+
+ static String keyspace = "keyspace1";
+
+ private Mutation mutation;
+ private MessageOut<Mutation> messageOut;
+
+ private ByteBuffer buffer;
+ private DataOutputBuffer outputBuffer;
+ private DataInputBuffer inputBuffer;
+
+
+ @State(Scope.Thread)
+ public static class ThreadState
+ {
+ MessageIn<Mutation> in;
+ int counter = 0;
+ }
+
+ @Setup
+ public void setup() throws IOException
+ {
+ Schema.instance.load(KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1)));
+ KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
+ CFMetaData metadata = CFMetaData.compile("CREATE TABLE userpics " +
+ "( userid bigint," +
+ "picid bigint," +
+ "commentid bigint, " +
+ "PRIMARY KEY(userid, picid))", keyspace);
+
+ Schema.instance.load(metadata);
+ Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.tables.with(metadata)));
+
+
+ mutation = (Mutation)UpdateBuilder.create(metadata, 1L).newRow(1L).add("commentid", 32L).makeMutation();
+ messageOut = mutation.createMessage();
+ buffer = ByteBuffer.allocate(messageOut.serializedSize(MessagingService.current_version));
+ outputBuffer = new DataOutputBufferFixed(buffer);
+ inputBuffer = new DataInputBuffer(buffer, false);
+
+ messageOut.serialize(outputBuffer, MessagingService.current_version);
+ }
+
+ @Benchmark
+ public void serialize(ThreadState state) throws IOException
+ {
+ buffer.rewind();
+ messageOut.serialize(outputBuffer, MessagingService.current_version);
+ state.counter++;
+ }
+
+ @Benchmark
+ public void deserialize(ThreadState state) throws IOException
+ {
+ buffer.rewind();
+ state.in = MessageIn.read(inputBuffer, MessagingService.current_version, 0);
+ state.counter++;
+ }
+
+ public static void main(String... args) throws Exception {
+ Options opts = new OptionsBuilder()
+ .include(".*"+MutationBench.class.getSimpleName()+".*")
+ .jvmArgs("-server")
+ .forks(1)
+ .mode(Mode.Throughput)
+ .addProfiler(StackProfiler.class)
+ .build();
+
+ Collection<RunResult> records = new Runner(opts).run();
+ for ( RunResult result : records) {
+ Result r = result.getPrimaryResult();
+ System.out.println("API replied benchmark score: "
+ + r.getScore() + " "
+ + r.getScoreUnit() + " over "
+ + r.getStatistics().getN() + " iterations");
+ }
+ }
+}
diff --git a/test/unit/org/apache/cassandra/MockSchema.java b/test/unit/org/apache/cassandra/MockSchema.java
index a406290..7f9de55 100644
--- a/test/unit/org/apache/cassandra/MockSchema.java
+++ b/test/unit/org/apache/cassandra/MockSchema.java
@@ -34,16 +34,14 @@
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.sstable.metadata.MetadataType;
import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.ChannelProxy;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.Memory;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.FileHandle;
import org.apache.cassandra.schema.CachingParams;
import org.apache.cassandra.schema.KeyspaceMetadata;
import org.apache.cassandra.schema.KeyspaceParams;
@@ -62,7 +60,7 @@
public static final Keyspace ks = Keyspace.mockKS(KeyspaceMetadata.create("mockks", KeyspaceParams.simpleTransient(1)));
public static final IndexSummary indexSummary;
- private static final SegmentedFile segmentedFile = new BufferedSegmentedFile(new ChannelProxy(temp("mocksegmentedfile")), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
+ private static final FileHandle RANDOM_ACCESS_READER_FACTORY = new FileHandle.Builder(temp("mocksegmentedfile").getAbsolutePath()).complete();
public static Memtable memtable(ColumnFamilyStore cfs)
{
@@ -89,7 +87,7 @@
Descriptor descriptor = new Descriptor(cfs.getDirectories().getDirectoryForNewSSTables(),
cfs.keyspace.getName(),
cfs.getColumnFamilyName(),
- generation);
+ generation, SSTableFormat.Type.BIG);
Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
for (Component component : components)
{
@@ -122,7 +120,7 @@
.finalizeMetadata(cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
.get(MetadataType.STATS);
SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata,
- segmentedFile.sharedCopy(), segmentedFile.sharedCopy(), indexSummary.sharedCopy(),
+ RANDOM_ACCESS_READER_FACTORY.sharedCopy(), RANDOM_ACCESS_READER_FACTORY.sharedCopy(), indexSummary.sharedCopy(),
new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL, header);
reader.first = reader.last = readerBounds(generation);
if (!keepRef)
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index 28fc8d5..81d25b9 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -271,6 +271,11 @@
MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(name, params, Tables.of(tables)), true);
}
+ public static void createKeyspace(String name, KeyspaceParams params, Tables tables, Types types)
+ {
+ MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(name, params, tables, Views.none(), types, Functions.none()), true);
+ }
+
public static ColumnDefinition integerColumn(String ksName, String cfName)
{
return new ColumnDefinition(ksName,
diff --git a/test/unit/org/apache/cassandra/UpdateBuilder.java b/test/unit/org/apache/cassandra/UpdateBuilder.java
index 3a5fbe6..047baaa 100644
--- a/test/unit/org/apache/cassandra/UpdateBuilder.java
+++ b/test/unit/org/apache/cassandra/UpdateBuilder.java
@@ -23,7 +23,6 @@
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.partitions.*;
import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.service.StorageService;
/**
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 7493b1f..3e8db41 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -57,7 +57,6 @@
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.CounterId;
import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.OpOrder;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
@@ -531,7 +530,7 @@
public static void spinAssertEquals(Object expected, Supplier<Object> s, int timeoutInSeconds)
{
long now = System.currentTimeMillis();
- while (System.currentTimeMillis() - now < now + (1000 * timeoutInSeconds))
+ while (System.currentTimeMillis() < now + (1000 * timeoutInSeconds))
{
if (s.get().equals(expected))
break;
diff --git a/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java b/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java
index 9943acb..f86df40 100644
--- a/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java
+++ b/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java
@@ -25,7 +25,6 @@
import java.util.stream.Collectors;
import javax.management.MalformedObjectNameException;
import javax.management.ObjectName;
-import javax.management.remote.JMXPrincipal;
import javax.security.auth.Subject;
import com.google.common.collect.ImmutableMap;
diff --git a/test/unit/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutorTest.java b/test/unit/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutorTest.java
index 46b2764..bf78d65 100644
--- a/test/unit/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutorTest.java
+++ b/test/unit/org/apache/cassandra/concurrent/DebuggableScheduledThreadPoolExecutorTest.java
@@ -20,7 +20,6 @@
import java.io.IOException;
import java.util.concurrent.CancellationException;
-import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
diff --git a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
index 989c524..78e0a6c 100644
--- a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
@@ -24,7 +24,6 @@
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.LivenessInfo;
import org.apache.cassandra.db.rows.BufferCell;
import org.apache.cassandra.db.rows.Cell;
import org.apache.cassandra.db.rows.CellPath;
diff --git a/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java b/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
index fe91650..93b16ce 100644
--- a/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
+++ b/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
@@ -23,8 +23,6 @@
import org.junit.Before;
import org.junit.Test;
-import org.apache.cassandra.io.util.RandomAccessReader;
-
public class LargeCompactValueTest extends CQLTester
{
@Before
diff --git a/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java b/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
index 37dc560..7ba9889 100644
--- a/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
+++ b/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
@@ -17,12 +17,6 @@
*/
package org.apache.cassandra.cql3;
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.util.Arrays;
-import java.util.Collections;
-
import org.junit.Test;
import static junit.framework.Assert.assertEquals;
diff --git a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
new file mode 100644
index 0000000..17e426b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.cql3.statements.ParsedStatement;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.schema.SchemaKeyspace;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.MD5Digest;
+
+public class PstmtPersistenceTest extends CQLTester
+{
+ @Test
+ public void testCachedPreparedStatements() throws Throwable
+ {
+ // need this for pstmt execution/validation tests
+ requireNetwork();
+
+ int rows = QueryProcessor.executeOnceInternal("SELECT * FROM " + SystemKeyspace.NAME + '.' + SystemKeyspace.PREPARED_STATEMENTS).size();
+ Assert.assertEquals(0, rows);
+
+ execute("CREATE KEYSPACE IF NOT EXISTS foo WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
+ execute("CREATE TABLE foo.bar (key text PRIMARY KEY, val int)");
+
+ ClientState clientState = ClientState.forExternalCalls(InetSocketAddress.createUnresolved("127.0.0.1", 1234));
+
+ createTable("CREATE TABLE %s (pk int PRIMARY KEY, val text)");
+
+ List<MD5Digest> stmtIds = new ArrayList<>();
+ // #0
+ stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + SchemaKeyspace.NAME + '.' + SchemaKeyspace.TABLES + " WHERE keyspace_name = ?", clientState, false).statementId);
+ // #1
+ stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + KEYSPACE + '.' + currentTable() + " WHERE pk = ?", clientState, false).statementId);
+ // #2
+ stmtIds.add(QueryProcessor.prepare("SELECT * FROM foo.bar WHERE key = ?", clientState, false).statementId);
+ clientState.setKeyspace("foo");
+ // #3
+ stmtIds.add(QueryProcessor.prepare("SELECT * FROM " + KEYSPACE + '.' + currentTable() + " WHERE pk = ?", clientState, false).statementId);
+ // #4
+ stmtIds.add(QueryProcessor.prepare("SELECT * FROM foo.bar WHERE key = ?", clientState, false).statementId);
+
+ Assert.assertEquals(5, stmtIds.size());
+ Assert.assertEquals(5, QueryProcessor.preparedStatementsCount());
+
+ String queryAll = "SELECT * FROM " + SystemKeyspace.NAME + '.' + SystemKeyspace.PREPARED_STATEMENTS;
+
+ rows = QueryProcessor.executeOnceInternal(queryAll).size();
+ Assert.assertEquals(5, rows);
+
+ QueryHandler handler = ClientState.getCQLQueryHandler();
+ validatePstmts(stmtIds, handler);
+
+ // clear prepared statements cache
+ QueryProcessor.clearPrepraredStatements();
+ Assert.assertEquals(0, QueryProcessor.preparedStatementsCount());
+ for (MD5Digest stmtId : stmtIds)
+ Assert.assertNull(handler.getPrepared(stmtId));
+
+ // load prepared statements and validate that these still execute fine
+ QueryProcessor.preloadPreparedStatement();
+ validatePstmts(stmtIds, handler);
+
+ // validate that the prepared statements are in the system table
+ for (UntypedResultSet.Row row : QueryProcessor.executeOnceInternal(queryAll))
+ {
+ MD5Digest digest = MD5Digest.wrap(ByteBufferUtil.getArray(row.getBytes("prepared_id")));
+ ParsedStatement.Prepared prepared = QueryProcessor.instance.getPrepared(digest);
+ Assert.assertNotNull(prepared);
+ }
+
+ // add anther prepared statement and sync it to table
+ QueryProcessor.prepare("SELECT * FROM bar WHERE key = ?", clientState, false);
+ Assert.assertEquals(6, QueryProcessor.preparedStatementsCount());
+ rows = QueryProcessor.executeOnceInternal(queryAll).size();
+ Assert.assertEquals(6, rows);
+
+ // drop a keyspace (prepared statements are removed - syncPreparedStatements() remove should the rows, too)
+ execute("DROP KEYSPACE foo");
+ Assert.assertEquals(3, QueryProcessor.preparedStatementsCount());
+ rows = QueryProcessor.executeOnceInternal(queryAll).size();
+ Assert.assertEquals(3, rows);
+
+ }
+
+ private void validatePstmts(List<MD5Digest> stmtIds, QueryHandler handler)
+ {
+ Assert.assertEquals(5, QueryProcessor.preparedStatementsCount());
+ QueryOptions optionsStr = QueryOptions.forInternalCalls(Collections.singletonList(UTF8Type.instance.fromString("foobar")));
+ QueryOptions optionsInt = QueryOptions.forInternalCalls(Collections.singletonList(Int32Type.instance.decompose(42)));
+ validatePstmt(handler, stmtIds.get(0), optionsStr);
+ validatePstmt(handler, stmtIds.get(1), optionsInt);
+ validatePstmt(handler, stmtIds.get(2), optionsStr);
+ validatePstmt(handler, stmtIds.get(3), optionsInt);
+ validatePstmt(handler, stmtIds.get(4), optionsStr);
+ }
+
+ private static void validatePstmt(QueryHandler handler, MD5Digest stmtId, QueryOptions options)
+ {
+ ParsedStatement.Prepared prepared = handler.getPrepared(stmtId);
+ Assert.assertNotNull(prepared);
+ handler.processPrepared(prepared.statement, QueryState.forInternalCalls(), options, Collections.emptyMap());
+ }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/SerializationMirrorTest.java b/test/unit/org/apache/cassandra/cql3/SerializationMirrorTest.java
index 49f77a7..3268784 100644
--- a/test/unit/org/apache/cassandra/cql3/SerializationMirrorTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SerializationMirrorTest.java
@@ -25,7 +25,6 @@
import org.junit.Test;
import junit.framework.Assert;
-import org.apache.cassandra.utils.ByteBufferUtil;
public class SerializationMirrorTest extends CQLTester
{
diff --git a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
index 22a4c49..c2de52a 100644
--- a/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
+++ b/test/unit/org/apache/cassandra/cql3/SimpleQueryTest.java
@@ -17,12 +17,8 @@
*/
package org.apache.cassandra.cql3;
-import java.util.*;
-
import org.junit.Test;
-import static junit.framework.Assert.*;
-
public class SimpleQueryTest extends CQLTester
{
@Test
diff --git a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index ff2af56..521c0a0 100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@ -22,10 +22,8 @@
import org.junit.Test;
-import com.sun.org.apache.xerces.internal.impl.xs.models.CMNodeFactory;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.marshal.BytesType;
import org.apache.cassandra.db.marshal.Int32Type;
diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
index 245ceb7..12cb673 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
@@ -28,7 +28,15 @@
import com.datastax.driver.core.exceptions.InvalidQueryException;
import junit.framework.Assert;
+import org.apache.cassandra.concurrent.SEPExecutor;
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.utils.FBUtilities;
public class ViewFilteringTest extends CQLTester
{
@@ -61,6 +69,16 @@
views.add(name);
}
+ private void updateView(String query, Object... params) throws Throwable
+ {
+ executeNet(protocolVersion, query, params);
+ while (!(((SEPExecutor) StageManager.getStage(Stage.VIEW_MUTATION)).getPendingTasks() == 0
+ && ((SEPExecutor) StageManager.getStage(Stage.VIEW_MUTATION)).getActiveCount() == 0))
+ {
+ Thread.sleep(1);
+ }
+ }
+
private void dropView(String name) throws Throwable
{
executeNet(protocolVersion, "DROP MATERIALIZED VIEW " + name);
@@ -1303,4 +1321,306 @@
executeNet(protocolVersion, "ALTER TABLE %s RENAME inetval TO foo");
assert !execute("SELECT * FROM mv_test").isEmpty();
}
+
+ @Test
+ public void testMVCreationWithNonPrimaryRestrictions() throws Throwable
+ {
+ createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b))");
+
+ execute("USE " + keyspace());
+ executeNet(protocolVersion, "USE " + keyspace());
+
+ try {
+ createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL AND d = 1 PRIMARY KEY (a, b, c)");
+ dropView("mv_test");
+ } catch(Exception e) {
+ throw new RuntimeException("MV creation with non primary column restrictions failed.", e);
+ }
+
+ dropTable("DROP TABLE %s");
+ }
+
+ @Test
+ public void testNonPrimaryRestrictions() throws Throwable
+ {
+ createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b))");
+
+ execute("USE " + keyspace());
+ executeNet(protocolVersion, "USE " + keyspace());
+
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 0, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 0);
+
+ // only accept rows where c = 1
+ createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND c = 1 PRIMARY KEY (a, b, c)");
+
+ while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test"))
+ Thread.sleep(10);
+
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0),
+ row(1, 0, 1, 0),
+ row(1, 1, 1, 0)
+ );
+
+ // insert new rows that do not match the filter
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, 2, 0);
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0),
+ row(1, 0, 1, 0),
+ row(1, 1, 1, 0)
+ );
+
+ // insert new row that does match the filter
+ execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 1, 0);
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0),
+ row(1, 0, 1, 0),
+ row(1, 1, 1, 0),
+ row(1, 2, 1, 0)
+ );
+
+ // update rows that don't match the filter
+ execute("UPDATE %s SET d = ? WHERE a = ? AND b = ?", 2, 2, 0);
+ execute("UPDATE %s SET d = ? WHERE a = ? AND b = ?", 1, 2, 1);
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0),
+ row(1, 0, 1, 0),
+ row(1, 1, 1, 0),
+ row(1, 2, 1, 0)
+ );
+
+ // update a row that does match the filter
+ execute("UPDATE %s SET d = ? WHERE a = ? AND b = ?", 1, 1, 0);
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0),
+ row(1, 0, 1, 1),
+ row(1, 1, 1, 0),
+ row(1, 2, 1, 0)
+ );
+
+ // delete rows that don't match the filter
+ execute("DELETE FROM %s WHERE a = ? AND b = ?", 2, 0);
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0),
+ row(1, 0, 1, 1),
+ row(1, 1, 1, 0),
+ row(1, 2, 1, 0)
+ );
+
+ // delete a row that does match the filter
+ execute("DELETE FROM %s WHERE a = ? AND b = ?", 1, 2);
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0),
+ row(1, 0, 1, 1),
+ row(1, 1, 1, 0)
+ );
+
+ // delete a partition that matches the filter
+ execute("DELETE FROM %s WHERE a = ?", 1);
+ assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+ row(0, 0, 1, 0),
+ row(0, 1, 1, 0)
+ );
+
+ dropView("mv_test");
+ dropTable("DROP TABLE %s");
+ }
+
+ @Test
+ public void complexRestrictedTimestampUpdateTestWithFlush() throws Throwable
+ {
+ complexRestrictedTimestampUpdateTest(true);
+ }
+
+ @Test
+ public void complexRestrictedTimestampUpdateTestWithoutFlush() throws Throwable
+ {
+ complexRestrictedTimestampUpdateTest(false);
+ }
+
+ public void complexRestrictedTimestampUpdateTest(boolean flush) throws Throwable
+ {
+ createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b))");
+
+ execute("USE " + keyspace());
+ executeNet(protocolVersion, "USE " + keyspace());
+ Keyspace ks = Keyspace.open(keyspace());
+
+ createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND c = 1 PRIMARY KEY (c, a, b)");
+ ks.getColumnFamilyStore("mv").disableAutoCompaction();
+
+ //Set initial values TS=0, matching the restriction and verify view
+ executeNet(protocolVersion, "INSERT INTO %s (a, b, c, d) VALUES (0, 0, 1, 0) USING TIMESTAMP 0");
+ assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
+
+ if (flush)
+ FBUtilities.waitOnFutures(ks.flush());
+
+ //update c's timestamp TS=2
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
+ assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
+
+ if (flush)
+ FBUtilities.waitOnFutures(ks.flush());
+
+ //change c's value and TS=3, tombstones c=1 and adds c=0 record
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? and b = ? ", 0, 0, 0);
+ assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 0, 0, 0));
+
+ if(flush)
+ {
+ ks.getColumnFamilyStore("mv").forceMajorCompaction();
+ FBUtilities.waitOnFutures(ks.flush());
+ }
+
+ //change c's value back to 1 with TS=4, check we can see d
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
+ if (flush)
+ {
+ ks.getColumnFamilyStore("mv").forceMajorCompaction();
+ FBUtilities.waitOnFutures(ks.flush());
+ }
+
+ assertRows(execute("SELECT d, e from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, null));
+
+
+ //Add e value @ TS=1
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 1 SET e = ? WHERE a = ? and b = ? ", 1, 0, 0);
+ assertRows(execute("SELECT d, e from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, 1));
+
+ if (flush)
+ FBUtilities.waitOnFutures(ks.flush());
+
+
+ //Change d value @ TS=2
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET d = ? WHERE a = ? and b = ? ", 2, 0, 0);
+ assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(2));
+
+ if (flush)
+ FBUtilities.waitOnFutures(ks.flush());
+
+
+ //Change d value @ TS=3
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? and b = ? ", 1, 0, 0);
+ assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(1));
+
+
+ //Tombstone c
+ executeNet(protocolVersion, "DELETE FROM %s WHERE a = ? and b = ?", 0, 0);
+ assertRows(execute("SELECT d from mv"));
+
+ //Add back without D
+ executeNet(protocolVersion, "INSERT INTO %s (a, b, c) VALUES (0, 0, 1)");
+
+ //Make sure D doesn't pop back in.
+ assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row((Object) null));
+
+
+ //New partition
+ // insert a row with timestamp 0
+ executeNet(protocolVersion, "INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP 0", 1, 0, 1, 0, 0);
+
+ // overwrite pk and e with timestamp 1, but don't overwrite d
+ executeNet(protocolVersion, "INSERT INTO %s (a, b, c, e) VALUES (?, ?, ?, ?) USING TIMESTAMP 1", 1, 0, 1, 0);
+
+ // delete with timestamp 0 (which should only delete d)
+ executeNet(protocolVersion, "DELETE FROM %s USING TIMESTAMP 0 WHERE a = ? AND b = ?", 1, 0);
+ assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 1, 1, 0),
+ row(1, 0, 1, null, 0)
+ );
+
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? AND b = ?", 1, 1, 1);
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? AND b = ?", 1, 1, 0);
+ assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 1, 1, 0),
+ row(1, 0, 1, null, 0)
+ );
+
+ executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? AND b = ?", 0, 1, 0);
+ assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 1, 1, 0),
+ row(1, 0, 1, 0, 0)
+ );
+ }
+
+ @Test
+ public void testRestrictedRegularColumnTimestampUpdates() throws Throwable
+ {
+ // Regression test for CASSANDRA-10910
+
+ createTable("CREATE TABLE %s (" +
+ "k int PRIMARY KEY, " +
+ "c int, " +
+ "val int)");
+
+ execute("USE " + keyspace());
+ executeNet(protocolVersion, "USE " + keyspace());
+
+ createView("mv_rctstest", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL AND c = 1 PRIMARY KEY (k,c)");
+
+ updateView("UPDATE %s SET c = ?, val = ? WHERE k = ?", 0, 0, 0);
+ updateView("UPDATE %s SET val = ? WHERE k = ?", 1, 0);
+ updateView("UPDATE %s SET c = ? WHERE k = ?", 1, 0);
+ assertRows(execute("SELECT c, k, val FROM mv_rctstest"), row(1, 0, 1));
+
+ updateView("TRUNCATE %s");
+
+ updateView("UPDATE %s USING TIMESTAMP 1 SET c = ?, val = ? WHERE k = ?", 0, 0, 0);
+ updateView("UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE k = ?", 1, 0);
+ updateView("UPDATE %s USING TIMESTAMP 2 SET val = ? WHERE k = ?", 1, 0);
+ updateView("UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE k = ?", 1, 0);
+ updateView("UPDATE %s USING TIMESTAMP 3 SET val = ? WHERE k = ?", 2, 0);
+ assertRows(execute("SELECT c, k, val FROM mv_rctstest"), row(1, 0, 2));
+ }
+
+ @Test
+ public void testOldTimestampsWithRestrictions() throws Throwable
+ {
+ createTable("CREATE TABLE %s (" +
+ "k int, " +
+ "c int, " +
+ "val text, " + "" +
+ "PRIMARY KEY(k, c))");
+
+ execute("USE " + keyspace());
+ executeNet(protocolVersion, "USE " + keyspace());
+
+ createView("mv_tstest", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL AND val = 'baz' PRIMARY KEY (val,k,c)");
+
+ for (int i = 0; i < 100; i++)
+ updateView("INSERT into %s (k,c,val)VALUES(?,?,?)", 0, i % 2, "baz");
+
+ Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush();
+
+ Assert.assertEquals(2, execute("select * from %s").size());
+ Assert.assertEquals(2, execute("select * from mv_tstest").size());
+
+ assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("baz"));
+ assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1));
+
+ //Make sure an old TS does nothing
+ updateView("UPDATE %s USING TIMESTAMP 100 SET val = ? where k = ? AND c = ?", "bar", 0, 1);
+ assertRows(execute("SELECT val from %s where k = 0 and c = 1"), row("baz"));
+ assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1));
+ assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"));
+
+ //Latest TS
+ updateView("UPDATE %s SET val = ? where k = ? AND c = ?", "bar", 0, 1);
+ assertRows(execute("SELECT val from %s where k = 0 and c = 1"), row("bar"));
+ assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"));
+ assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0));
+ }
}
diff --git a/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java b/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
index 065fdbd..a07f8f9 100644
--- a/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/selection/TermSelectionTest.java
@@ -19,7 +19,6 @@
package org.apache.cassandra.cql3.selection;
import java.math.BigDecimal;
-import java.math.BigInteger;
import java.util.*;
import org.junit.Test;
@@ -29,7 +28,6 @@
import org.apache.cassandra.transport.messages.ResultMessage;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
public class TermSelectionTest extends CQLTester
{
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
index 33b4a4f..a09d0d7 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
@@ -21,7 +21,6 @@
import org.junit.Test;
import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.exceptions.InvalidRequestException;
public class CountersTest extends CQLTester
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
index 0e255e4..a14e4a5 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
@@ -794,6 +794,52 @@
}
@Test
+ public void testInsertJsonSyntaxDefaultUnset() throws Throwable
+ {
+ createTable("CREATE TABLE %s (k int primary key, v1 int, v2 int)");
+ execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v1\": 0, \"v2\": 0}");
+
+ // leave v1 unset
+ execute("INSERT INTO %s JSON ? DEFAULT UNSET", "{\"k\": 0, \"v2\": 2}");
+ assertRows(execute("SELECT * FROM %s"),
+ row(0, 0, 2)
+ );
+
+ // explicit specification DEFAULT NULL
+ execute("INSERT INTO %s JSON ? DEFAULT NULL", "{\"k\": 0, \"v2\": 2}");
+ assertRows(execute("SELECT * FROM %s"),
+ row(0, null, 2)
+ );
+
+ // implicitly setting v2 to null
+ execute("INSERT INTO %s JSON ? DEFAULT NULL", "{\"k\": 0}");
+ assertRows(execute("SELECT * FROM %s"),
+ row(0, null, null)
+ );
+
+ // mix setting null explicitly with default unset:
+ // set values for all fields
+ execute("INSERT INTO %s JSON ?", "{\"k\": 1, \"v1\": 1, \"v2\": 1}");
+ // explicitly set v1 to null while leaving v2 unset which retains its value
+ execute("INSERT INTO %s JSON ? DEFAULT UNSET", "{\"k\": 1, \"v1\": null}");
+ assertRows(execute("SELECT * FROM %s WHERE k=1"),
+ row(1, null, 1)
+ );
+
+ // test string literal instead of bind marker
+ execute("INSERT INTO %s JSON '{\"k\": 2, \"v1\": 2, \"v2\": 2}'");
+ // explicitly set v1 to null while leaving v2 unset which retains its value
+ execute("INSERT INTO %s JSON '{\"k\": 2, \"v1\": null}' DEFAULT UNSET");
+ assertRows(execute("SELECT * FROM %s WHERE k=2"),
+ row(2, null, 2)
+ );
+ execute("INSERT INTO %s JSON '{\"k\": 2}' DEFAULT NULL");
+ assertRows(execute("SELECT * FROM %s WHERE k=2"),
+ row(2, null, null)
+ );
+ }
+
+ @Test
public void testCaseSensitivity() throws Throwable
{
createTable("CREATE TABLE %s (k int primary key, \"Foo\" int)");
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
index d085a9d..a3a8e37 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -36,7 +36,6 @@
import org.apache.cassandra.cql3.statements.ModificationStatement;
import org.apache.cassandra.exceptions.*;
import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.utils.Pair;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallClone.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallClone.java
index e8bae70..9efa83a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallClone.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallClone.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
try
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallComDatastax.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallComDatastax.java
index 1af5b01..4555ff5 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallComDatastax.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallComDatastax.java
@@ -36,6 +36,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
DataType.cint();
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallFinalize.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallFinalize.java
index 5208849..b1ec15f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallFinalize.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallFinalize.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
try
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallOrgApache.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallOrgApache.java
index 758d0d0..728e482 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallOrgApache.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/CallOrgApache.java
@@ -36,6 +36,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
DatabaseDescriptor.getClusterName();
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithField.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithField.java
index 256c2bd..4c38b44 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithField.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithField.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer.java
index 3366314..cc2738a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer2.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer2.java
index aaf3e7b..780c0e4 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer2.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer2.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer3.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer3.java
index 4895aa0..e163ec9 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer3.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInitializer3.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass.java
index 2166771..3c4dc9b 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass2.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass2.java
index 9c18510..b316040 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass2.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithInnerClass2.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
// this is fine
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInitializer.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInitializer.java
index 3c958e8..c97a94a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInitializer.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInitializer.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInnerClass.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInnerClass.java
index fada145..1b019cc 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInnerClass.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/ClassWithStaticInnerClass.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/GoodClass.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/GoodClass.java
index eb25f72..54821b9 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/GoodClass.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/GoodClass.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
return null;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronized.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronized.java
index bbbc823..dba846d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronized.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronized.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
synchronized (this)
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotify.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotify.java
index 07c70c7..63c319c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotify.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotify.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
synchronized (this)
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotifyAll.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotifyAll.java
index 529c995..4d0c2a0 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotifyAll.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithNotifyAll.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
synchronized (this)
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWait.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWait.java
index 6e39813..b002086 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWait.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWait.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
synchronized (this)
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitL.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitL.java
index ac29211..f128fac 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitL.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitL.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
synchronized (this)
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitLI.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitLI.java
index 3b9ce8b..d439518 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitLI.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UseOfSynchronizedWithWaitLI.java
@@ -35,6 +35,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
synchronized (this)
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UsingMapEntry.java b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UsingMapEntry.java
index 5091dc1..b99dbfd 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UsingMapEntry.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/udfverify/UsingMapEntry.java
@@ -37,6 +37,11 @@
super(returnDataType, argDataTypes, udfContext);
}
+ protected Object executeAggregateImpl(int protocolVersion, Object firstParam, List<ByteBuffer> params)
+ {
+ throw new UnsupportedOperationException();
+ }
+
protected ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
{
Map<String, String> map = new HashMap<>();
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
index 9733eb2..49bfe88 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/OverflowTest.java
@@ -21,7 +21,6 @@
import org.junit.Test;
-import static junit.framework.Assert.assertFalse;
import static junit.framework.Assert.assertNull;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index 24a9528..506d533 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -18,7 +18,6 @@
package org.apache.cassandra.cql3.validation.operations;
import java.math.BigDecimal;
-import java.math.MathContext;
import java.math.RoundingMode;
import java.nio.ByteBuffer;
import java.text.SimpleDateFormat;
@@ -40,6 +39,8 @@
import ch.qos.logback.classic.spi.TurboFilterList;
import ch.qos.logback.classic.turbo.ReconfigureOnChangeFilter;
import ch.qos.logback.classic.turbo.TurboFilter;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.TupleValue;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.CQLTester;
import org.apache.cassandra.cql3.QueryProcessor;
@@ -48,7 +49,6 @@
import org.apache.cassandra.cql3.functions.UDAggregate;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.DynamicCompositeType;
import org.apache.cassandra.db.marshal.TypeParser;
import org.apache.cassandra.exceptions.FunctionExecutionException;
import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -1883,6 +1883,7 @@
row(finalFunc, initCond));
}
+ @Test
public void testCustomTypeInitcond() throws Throwable
{
try
@@ -1966,4 +1967,58 @@
assertRows(execute("select avg(val) from %s where bucket in (1, 2, 3);"),
row(a));
}
+
+ @Test
+ public void testSameStateInstance() throws Throwable
+ {
+ // CASSANDRA-9613 removes the neccessity to re-serialize the state variable for each
+ // UDA state function and final function call.
+ //
+ // To test that the same state object instance is used during each invocation of the
+ // state and final function, this test uses a trick:
+ // it puts the identity hash code of the state variable to a tuple. The test then
+ // just asserts that the identity hash code is the same for all invocations
+ // of the state function and the final function.
+
+ String sf = createFunction(KEYSPACE,
+ "tuple<int,int,int,int>, int",
+ "CREATE FUNCTION %s(s tuple<int,int,int,int>, i int) " +
+ "CALLED ON NULL INPUT " +
+ "RETURNS tuple<int,int,int,int> " +
+ "LANGUAGE java " +
+ "AS 's.setInt(i, System.identityHashCode(s)); return s;'");
+
+ String ff = createFunction(KEYSPACE,
+ "tuple<int,int,int,int>",
+ "CREATE FUNCTION %s(s tuple<int,int,int,int>) " +
+ "CALLED ON NULL INPUT " +
+ "RETURNS tuple<int,int,int,int> " +
+ "LANGUAGE java " +
+ "AS 's.setInt(3, System.identityHashCode(s)); return s;'");
+
+ String a = createAggregate(KEYSPACE,
+ "int",
+ "CREATE AGGREGATE %s(int) " +
+ "SFUNC " + shortFunctionName(sf) + ' ' +
+ "STYPE tuple<int,int,int,int> " +
+ "FINALFUNC " + shortFunctionName(ff) + ' ' +
+ "INITCOND (0,1,2)");
+
+ createTable("CREATE TABLE %s (a int primary key, b int)");
+ execute("INSERT INTO %s (a, b) VALUES (0, 0)");
+ execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+ execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+ try (Session s = sessionNet())
+ {
+ com.datastax.driver.core.Row row = s.execute("SELECT " + a + "(b) FROM " + KEYSPACE + '.' + currentTable()).one();
+ TupleValue tuple = row.getTupleValue(0);
+ int h0 = tuple.getInt(0);
+ int h1 = tuple.getInt(1);
+ int h2 = tuple.getInt(2);
+ int h3 = tuple.getInt(3);
+ assertEquals(h0, h1);
+ assertEquals(h0, h2);
+ assertEquals(h0, h3);
+ }
+ }
}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index bb4bf48..092efc2 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -99,6 +99,76 @@
}
@Test
+ public void testDropWithTimestamp() throws Throwable
+ {
+ createTable("CREATE TABLE %s (id int, c1 int, v1 int, todrop int, PRIMARY KEY (id, c1));");
+ for (int i = 0; i < 5; i++)
+ execute("INSERT INTO %s (id, c1, v1, todrop) VALUES (?, ?, ?, ?) USING TIMESTAMP ?", 1, i, i, i, 10000L * i);
+
+ // flush is necessary since otherwise the values of `todrop` will get discarded during
+ // alter statement
+ flush(true);
+ execute("ALTER TABLE %s DROP todrop USING TIMESTAMP 20000;");
+ execute("ALTER TABLE %s ADD todrop int;");
+ execute("INSERT INTO %s (id, c1, v1, todrop) VALUES (?, ?, ?, ?) USING TIMESTAMP ?", 1, 100, 100, 100, 30000L);
+ assertRows(execute("SELECT id, c1, v1, todrop FROM %s"),
+ row(1, 0, 0, null),
+ row(1, 1, 1, null),
+ row(1, 2, 2, null),
+ row(1, 3, 3, 3),
+ row(1, 4, 4, 4),
+ row(1, 100, 100, 100));
+ }
+
+ @Test
+ public void testDropStaticWithTimestamp() throws Throwable
+ {
+ createTable("CREATE TABLE %s (id int, c1 int, v1 int, todrop int static, PRIMARY KEY (id, c1));");
+ for (int i = 0; i < 5; i++)
+ execute("INSERT INTO %s (id, c1, v1, todrop) VALUES (?, ?, ?, ?) USING TIMESTAMP ?", 1, i, i, i, 10000L * i);
+
+ // flush is necessary since otherwise the values of `todrop` will get discarded during
+ // alter statement
+ flush(true);
+ execute("ALTER TABLE %s DROP todrop USING TIMESTAMP 20000;");
+ execute("ALTER TABLE %s ADD todrop int static;");
+ execute("INSERT INTO %s (id, c1, v1, todrop) VALUES (?, ?, ?, ?) USING TIMESTAMP ?", 1, 100, 100, 100, 30000L);
+ // static column value with largest timestmap will be available again
+ assertRows(execute("SELECT id, c1, v1, todrop FROM %s"),
+ row(1, 0, 0, 4),
+ row(1, 1, 1, 4),
+ row(1, 2, 2, 4),
+ row(1, 3, 3, 4),
+ row(1, 4, 4, 4),
+ row(1, 100, 100, 4));
+ }
+
+ @Test
+ public void testDropMultipleWithTimestamp() throws Throwable
+ {
+ createTable("CREATE TABLE %s (id int, c1 int, v1 int, todrop1 int, todrop2 int, PRIMARY KEY (id, c1));");
+ for (int i = 0; i < 5; i++)
+ execute("INSERT INTO %s (id, c1, v1, todrop1, todrop2) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP ?", 1, i, i, i, i, 10000L * i);
+
+ // flush is necessary since otherwise the values of `todrop1` and `todrop2` will get discarded during
+ // alter statement
+ flush(true);
+ execute("ALTER TABLE %s DROP (todrop1, todrop2) USING TIMESTAMP 20000;");
+ execute("ALTER TABLE %s ADD todrop1 int;");
+ execute("ALTER TABLE %s ADD todrop2 int;");
+
+ execute("INSERT INTO %s (id, c1, v1, todrop1, todrop2) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP ?", 1, 100, 100, 100, 100, 40000L);
+ assertRows(execute("SELECT id, c1, v1, todrop1, todrop2 FROM %s"),
+ row(1, 0, 0, null, null),
+ row(1, 1, 1, null, null),
+ row(1, 2, 2, null, null),
+ row(1, 3, 3, 3, 3),
+ row(1, 4, 4, 4, 4),
+ row(1, 100, 100, 100, 100));
+ }
+
+
+ @Test
public void testChangeStrategyWithUnquotedAgrument() throws Throwable
{
createTable("CREATE TABLE %s (id text PRIMARY KEY);");
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
index b132958..9bb2468 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -27,7 +27,6 @@
import static java.lang.String.format;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
public class InsertUpdateIfConditionTest extends CQLTester
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
new file mode 100644
index 0000000..df8fe49
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
@@ -0,0 +1,683 @@
+/*
+ * 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;
+
+import java.io.FileReader;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.*;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Files;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.*;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.statements.*;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.index.sasi.*;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.utils.*;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class ColumnFamilyStoreCQLHelperTest extends CQLTester
+{
+ @Before
+ public void defineSchema() throws ConfigurationException
+ {
+ SchemaLoader.prepareServer();
+ }
+
+ @Test
+ public void testUserTypesCQL()
+ {
+ String keyspace = "cql_test_keyspace_user_types";
+ String table = "test_table_user_types";
+
+ UserType typeA = new UserType(keyspace, ByteBufferUtil.bytes("a"),
+ Arrays.asList(FieldIdentifier.forUnquoted("a1"),
+ FieldIdentifier.forUnquoted("a2"),
+ FieldIdentifier.forUnquoted("a3")),
+ Arrays.asList(IntegerType.instance,
+ IntegerType.instance,
+ IntegerType.instance),
+ true);
+
+ UserType typeB = new UserType(keyspace, ByteBufferUtil.bytes("b"),
+ Arrays.asList(FieldIdentifier.forUnquoted("b1"),
+ FieldIdentifier.forUnquoted("b2"),
+ FieldIdentifier.forUnquoted("b3")),
+ Arrays.asList(typeA,
+ typeA,
+ typeA),
+ true);
+
+ UserType typeC = new UserType(keyspace, ByteBufferUtil.bytes("c"),
+ Arrays.asList(FieldIdentifier.forUnquoted("c1"),
+ FieldIdentifier.forUnquoted("c2"),
+ FieldIdentifier.forUnquoted("c3")),
+ Arrays.asList(typeB,
+ typeB,
+ typeB),
+ true);
+
+ CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addClusteringColumn("ck1", IntegerType.instance)
+ .addRegularColumn("reg1", typeC)
+ .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false))
+ .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true))
+ .build();
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ Tables.of(cfm),
+ Types.of(typeA, typeB, typeC));
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ assertEquals(ImmutableList.of("CREATE TYPE cql_test_keyspace_user_types.a(a1 varint, a2 varint, a3 varint);",
+ "CREATE TYPE cql_test_keyspace_user_types.b(b1 a, b2 a, b3 a);",
+ "CREATE TYPE cql_test_keyspace_user_types.c(c1 b, c2 b, c3 b);"),
+ ColumnFamilyStoreCQLHelper.getUserTypesAsCQL(cfs.metadata));
+ }
+
+ @Test
+ public void testDroppedColumnsCQL()
+ {
+ String keyspace = "cql_test_keyspace_dropped_columns";
+ String table = "test_table_dropped_columns";
+
+ CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addClusteringColumn("ck1", IntegerType.instance)
+ .addRegularColumn("reg1", IntegerType.instance)
+ .addRegularColumn("reg2", IntegerType.instance)
+ .addRegularColumn("reg3", IntegerType.instance)
+ .build();
+
+
+ ColumnDefinition reg1 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1"));
+ ColumnDefinition reg2 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg2"));
+ ColumnDefinition reg3 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg3"));
+
+ cfm.removeColumnDefinition(reg1);
+ cfm.removeColumnDefinition(reg2);
+ cfm.removeColumnDefinition(reg3);
+
+ cfm.recordColumnDrop(reg1, 10000);
+ cfm.recordColumnDrop(reg2, 20000);
+ cfm.recordColumnDrop(reg3, 30000);
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ cfm);
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ assertEquals(ImmutableList.of("ALTER TABLE cql_test_keyspace_dropped_columns.test_table_dropped_columns DROP reg1 USING TIMESTAMP 10000;",
+ "ALTER TABLE cql_test_keyspace_dropped_columns.test_table_dropped_columns DROP reg3 USING TIMESTAMP 30000;",
+ "ALTER TABLE cql_test_keyspace_dropped_columns.test_table_dropped_columns DROP reg2 USING TIMESTAMP 20000;"),
+ ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(cfs.metadata));
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS cql_test_keyspace_dropped_columns.test_table_dropped_columns (\n" +
+ "\tpk1 varint,\n" +
+ "\tck1 varint,\n" +
+ "\treg1 varint,\n" +
+ "\treg3 varint,\n" +
+ "\treg2 varint,\n" +
+ "\tPRIMARY KEY (pk1, ck1))"));
+ }
+
+ @Test
+ public void testReaddedColumns()
+ {
+ String keyspace = "cql_test_keyspace_readded_columns";
+ String table = "test_table_readded_columns";
+
+ CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addClusteringColumn("ck1", IntegerType.instance)
+ .addRegularColumn("reg1", IntegerType.instance)
+ .addStaticColumn("reg2", IntegerType.instance)
+ .addRegularColumn("reg3", IntegerType.instance)
+ .build();
+
+ ColumnDefinition reg1 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1"));
+ ColumnDefinition reg2 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg2"));
+
+ cfm.removeColumnDefinition(reg1);
+ cfm.removeColumnDefinition(reg2);
+
+ cfm.recordColumnDrop(reg1, 10000);
+ cfm.recordColumnDrop(reg2, 20000);
+
+ cfm.addColumnDefinition(reg1);
+ cfm.addColumnDefinition(reg2);
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ cfm);
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ // when re-adding, column is present in CREATE, then in DROP and then in ADD again, to record DROP with a proper timestamp
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS cql_test_keyspace_readded_columns.test_table_readded_columns (\n" +
+ "\tpk1 varint,\n" +
+ "\tck1 varint,\n" +
+ "\treg2 varint static,\n" +
+ "\treg1 varint,\n" +
+ "\treg3 varint,\n" +
+ "\tPRIMARY KEY (pk1, ck1))"));
+
+ assertEquals(ImmutableList.of("ALTER TABLE cql_test_keyspace_readded_columns.test_table_readded_columns DROP reg1 USING TIMESTAMP 10000;",
+ "ALTER TABLE cql_test_keyspace_readded_columns.test_table_readded_columns ADD reg1 varint;",
+ "ALTER TABLE cql_test_keyspace_readded_columns.test_table_readded_columns DROP reg2 USING TIMESTAMP 20000;",
+ "ALTER TABLE cql_test_keyspace_readded_columns.test_table_readded_columns ADD reg2 varint static;"),
+ ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(cfs.metadata));
+ }
+
+ @Test
+ public void testCfmColumnsCQL()
+ {
+ String keyspace = "cql_test_keyspace_create_table";
+ String table = "test_table_create_table";
+
+ CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addPartitionKey("pk2", AsciiType.instance)
+ .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
+ .addClusteringColumn("ck2", IntegerType.instance)
+ .addStaticColumn("st1", AsciiType.instance)
+ .addRegularColumn("reg1", AsciiType.instance)
+ .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false))
+ .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true))
+ .build();
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ cfm);
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS cql_test_keyspace_create_table.test_table_create_table (\n" +
+ "\tpk1 varint,\n" +
+ "\tpk2 ascii,\n" +
+ "\tck1 varint,\n" +
+ "\tck2 varint,\n" +
+ "\tst1 ascii static,\n" +
+ "\treg1 ascii,\n" +
+ "\treg2 frozen<list<varint>>,\n" +
+ "\treg3 map<ascii, varint>,\n" +
+ "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
+ }
+
+ @Test
+ public void testCfmCompactStorageCQL()
+ {
+ String keyspace = "cql_test_keyspace_compact";
+ String table = "test_table_compact";
+
+ CFMetaData cfm = CFMetaData.Builder.createDense(keyspace, table, true, false)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addPartitionKey("pk2", AsciiType.instance)
+ .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
+ .addClusteringColumn("ck2", IntegerType.instance)
+ .addRegularColumn("reg", IntegerType.instance)
+ .build();
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ cfm);
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS cql_test_keyspace_compact.test_table_compact (\n" +
+ "\tpk1 varint,\n" +
+ "\tpk2 ascii,\n" +
+ "\tck1 varint,\n" +
+ "\tck2 varint,\n" +
+ "\treg varint,\n" +
+ "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
+ "\tWITH ID = " + cfm.cfId + "\n" +
+ "\tAND COMPACT STORAGE\n" +
+ "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
+ }
+
+ @Test
+ public void testCfmCounterCQL()
+ {
+ String keyspace = "cql_test_keyspace_counter";
+ String table = "test_table_counter";
+
+ CFMetaData cfm = CFMetaData.Builder.createDense(keyspace, table, true, true)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addPartitionKey("pk2", AsciiType.instance)
+ .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
+ .addClusteringColumn("ck2", IntegerType.instance)
+ .addRegularColumn("cnt", CounterColumnType.instance)
+ .build();
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ cfm);
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS cql_test_keyspace_counter.test_table_counter (\n" +
+ "\tpk1 varint,\n" +
+ "\tpk2 ascii,\n" +
+ "\tck1 varint,\n" +
+ "\tck2 varint,\n" +
+ "\tcnt counter,\n" +
+ "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
+ "\tWITH ID = " + cfm.cfId + "\n" +
+ "\tAND COMPACT STORAGE\n" +
+ "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
+ }
+
+ @Test
+ public void testCfmOptionsCQL()
+ {
+ String keyspace = "cql_test_keyspace_options";
+ String table = "test_table_options";
+
+ CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addClusteringColumn("cl1", IntegerType.instance)
+ .addRegularColumn("reg1", AsciiType.instance)
+ .build();
+
+ cfm.recordColumnDrop(cfm.getColumnDefinition(ByteBuffer.wrap("reg1".getBytes())), FBUtilities.timestampMicros());
+ cfm.bloomFilterFpChance(1.0);
+ cfm.comment("comment");
+ cfm.compaction(CompactionParams.lcs(Collections.singletonMap("sstable_size_in_mb", "1")));
+ cfm.compression(CompressionParams.lz4(1 << 16));
+ cfm.dcLocalReadRepairChance(0.2);
+ cfm.crcCheckChance(0.3);
+ cfm.defaultTimeToLive(4);
+ cfm.gcGraceSeconds(5);
+ cfm.minIndexInterval(6);
+ cfm.maxIndexInterval(7);
+ cfm.memtableFlushPeriod(8);
+ cfm.readRepairChance(0.9);
+ cfm.speculativeRetry(SpeculativeRetryParam.always());
+ cfm.extensions(ImmutableMap.of("ext1",
+ ByteBuffer.wrap("val1".getBytes())));
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ cfm);
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).endsWith(
+ "AND bloom_filter_fp_chance = 1.0\n" +
+ "\tAND dclocal_read_repair_chance = 0.2\n" +
+ "\tAND crc_check_chance = 0.3\n" +
+ "\tAND default_time_to_live = 4\n" +
+ "\tAND gc_grace_seconds = 5\n" +
+ "\tAND min_index_interval = 6\n" +
+ "\tAND max_index_interval = 7\n" +
+ "\tAND memtable_flush_period_in_ms = 8\n" +
+ "\tAND read_repair_chance = 0.9\n" +
+ "\tAND speculative_retry = 'ALWAYS'\n" +
+ "\tAND comment = 'comment'\n" +
+ "\tAND caching = { 'keys': 'ALL', 'rows_per_partition': 'NONE' }\n" +
+ "\tAND compaction = { 'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', 'sstable_size_in_mb': '1' }\n" +
+ "\tAND compression = { 'chunk_length_in_kb': '64', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor' }\n" +
+ "\tAND cdc = false\n" +
+ "\tAND extensions = { 'ext1': 0x76616c31 };"
+ ));
+ }
+
+ @Test
+ public void testCfmIndexJson()
+ {
+ String keyspace = "cql_test_keyspace_3";
+ String table = "test_table_3";
+
+ CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
+ .addPartitionKey("pk1", IntegerType.instance)
+ .addClusteringColumn("cl1", IntegerType.instance)
+ .addRegularColumn("reg1", AsciiType.instance)
+ .build();
+
+ cfm.indexes(cfm.getIndexes()
+ .with(IndexMetadata.fromIndexTargets(cfm,
+ Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
+ IndexTarget.Type.VALUES)),
+ "indexName",
+ IndexMetadata.Kind.COMPOSITES,
+ Collections.emptyMap()))
+ .with(IndexMetadata.fromIndexTargets(cfm,
+ Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
+ IndexTarget.Type.KEYS)),
+ "indexName2",
+ IndexMetadata.Kind.COMPOSITES,
+ Collections.emptyMap()))
+ .with(IndexMetadata.fromIndexTargets(cfm,
+ Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
+ IndexTarget.Type.KEYS_AND_VALUES)),
+ "indexName3",
+ IndexMetadata.Kind.COMPOSITES,
+ Collections.emptyMap()))
+ .with(IndexMetadata.fromIndexTargets(cfm,
+ Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
+ IndexTarget.Type.KEYS_AND_VALUES)),
+ "indexName4",
+ IndexMetadata.Kind.CUSTOM,
+ Collections.singletonMap(IndexTarget.CUSTOM_INDEX_OPTION_NAME,
+ SASIIndex.class.getName()))
+ ));
+
+
+ SchemaLoader.createKeyspace(keyspace,
+ KeyspaceParams.simple(1),
+ cfm);
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
+
+ assertEquals(ImmutableList.of("CREATE INDEX \"indexName\" ON cql_test_keyspace_3.test_table_3 (reg1);",
+ "CREATE INDEX \"indexName2\" ON cql_test_keyspace_3.test_table_3 (reg1);",
+ "CREATE INDEX \"indexName3\" ON cql_test_keyspace_3.test_table_3 (reg1);",
+ "CREATE CUSTOM INDEX \"indexName4\" ON cql_test_keyspace_3.test_table_3 (reg1) USING 'org.apache.cassandra.index.sasi.SASIIndex';"),
+ ColumnFamilyStoreCQLHelper.getIndexesAsCQL(cfs.metadata));
+ }
+
+ private final static String SNAPSHOT = "testsnapshot";
+
+ @Test
+ public void testSnapshot() throws Throwable
+ {
+ String typeA = createType("CREATE TYPE %s (a1 varint, a2 varint, a3 varint);");
+ String typeB = createType("CREATE TYPE %s (b1 frozen<" + typeA + ">, b2 frozen<" + typeA + ">, b3 frozen<" + typeA + ">);");
+ String typeC = createType("CREATE TYPE %s (c1 frozen<" + typeB + ">, c2 frozen<" + typeB + ">, c3 frozen<" + typeB + ">);");
+
+ String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
+ "pk1 varint," +
+ "pk2 ascii," +
+ "ck1 varint," +
+ "ck2 varint," +
+ "reg1 " + typeC + "," +
+ "reg2 int," +
+ "reg3 int," +
+ "PRIMARY KEY ((pk1, pk2), ck1, ck2)) WITH " +
+ "CLUSTERING ORDER BY (ck1 ASC, ck2 DESC);");
+
+ alterTable("ALTER TABLE %s DROP reg3 USING TIMESTAMP 10000;");
+ alterTable("ALTER TABLE %s ADD reg3 int;");
+
+ for (int i = 0; i < 10; i++)
+ execute("INSERT INTO %s (pk1, pk2, ck1, ck2, reg1, reg2) VALUES (?, ?, ?, ?, ?, ?)", i, i + 1, i + 2, i + 3, null, i + 5);
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
+ cfs.snapshot(SNAPSHOT);
+
+ String schema = Files.toString(cfs.getDirectories().getSnapshotSchemaFile(SNAPSHOT), Charset.defaultCharset());
+ assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(a1 varint, a2 varint, a3 varint);", keyspace(), typeA)));
+ assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(a1 varint, a2 varint, a3 varint);", keyspace(), typeA)));
+ assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(b1 frozen<%s>, b2 frozen<%s>, b3 frozen<%s>);", keyspace(), typeB, typeA, typeA, typeA)));
+ assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(c1 frozen<%s>, c2 frozen<%s>, c3 frozen<%s>);", keyspace(), typeC, typeB, typeB, typeB)));
+
+ schema = schema.substring(schema.indexOf("CREATE TABLE")); // trim to ensure order
+
+ assertTrue(schema.startsWith("CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
+ "\tpk1 varint,\n" +
+ "\tpk2 ascii,\n" +
+ "\tck1 varint,\n" +
+ "\tck2 varint,\n" +
+ "\treg2 int,\n" +
+ "\treg3 int,\n" +
+ "\treg1 " + typeC + ",\n" +
+ "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND CLUSTERING ORDER BY (ck1 ASC, ck2 DESC)"));
+
+ schema = schema.substring(schema.indexOf("ALTER"));
+ assertTrue(schema.startsWith(String.format("ALTER TABLE %s.%s DROP reg3 USING TIMESTAMP 10000;", keyspace(), tableName)));
+ assertTrue(schema.contains(String.format("ALTER TABLE %s.%s ADD reg3 int;", keyspace(), tableName)));
+
+ JSONObject manifest = (JSONObject) new JSONParser().parse(new FileReader(cfs.getDirectories().getSnapshotManifestFile(SNAPSHOT)));
+ JSONArray files = (JSONArray) manifest.get("files");
+ Assert.assertEquals(1, files.size());
+ }
+
+ @Test
+ public void testDroppedType() throws Throwable
+ {
+ String typeA = createType("CREATE TYPE %s (a1 varint, a2 varint, a3 varint);");
+ String typeB = createType("CREATE TYPE %s (b1 frozen<" + typeA + ">, b2 frozen<" + typeA + ">, b3 frozen<" + typeA + ">);");
+
+ String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
+ "pk1 varint," +
+ "ck1 varint," +
+ "reg1 " + typeB + "," +
+ "reg2 varint," +
+ "PRIMARY KEY (pk1, ck1));");
+
+ alterTable("ALTER TABLE %s DROP reg1 USING TIMESTAMP 10000;");
+
+ Runnable validate = () -> {
+ try
+ {
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
+ cfs.snapshot(SNAPSHOT);
+ String schema = Files.toString(cfs.getDirectories().getSnapshotSchemaFile(SNAPSHOT), Charset.defaultCharset());
+
+ // When both column and it's type are dropped, the type in column definition gets substituted with a tuple
+ assertTrue(schema.startsWith("CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
+ "\tpk1 varint,\n" +
+ "\tck1 varint,\n" +
+ "\treg2 varint,\n" +
+ "\treg1 frozen<tuple<frozen<tuple<varint, varint, varint>>, frozen<tuple<varint, varint, varint>>, frozen<tuple<varint, varint, varint>>>>,\n" +
+ "\tPRIMARY KEY (pk1, ck1))"));
+ assertTrue(schema.contains("ALTER TABLE " + keyspace() + "." + tableName + " DROP reg1 USING TIMESTAMP 10000;"));
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException(e);
+ }
+ };
+
+ // Validate before and after the type drop
+ validate.run();
+ schemaChange("DROP TYPE " + keyspace() + "." + typeB);
+ schemaChange("DROP TYPE " + keyspace() + "." + typeA);
+ validate.run();
+ }
+
+ @Test
+ public void testDenseTable() throws Throwable
+ {
+ String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
+ "pk1 varint PRIMARY KEY," +
+ "reg1 int)" +
+ " WITH COMPACT STORAGE");
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
+ "\tpk1 varint PRIMARY KEY,\n" +
+ "\treg1 int)\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND COMPACT STORAGE"));
+ }
+
+ @Test
+ public void testStaticCompactTable() throws Throwable
+ {
+ String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
+ "pk1 varint PRIMARY KEY," +
+ "reg1 int," +
+ "reg2 int)" +
+ " WITH COMPACT STORAGE");
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
+ "\tpk1 varint PRIMARY KEY,\n" +
+ "\treg1 int,\n" +
+ "\treg2 int)\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND COMPACT STORAGE"));
+ }
+
+ @Test
+ public void testStaticCompactWithCounters() throws Throwable
+ {
+ String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
+ "pk1 varint PRIMARY KEY," +
+ "reg1 counter," +
+ "reg2 counter)" +
+ " WITH COMPACT STORAGE");
+
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
+ "\tpk1 varint PRIMARY KEY,\n" +
+ "\treg1 counter,\n" +
+ "\treg2 counter)\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND COMPACT STORAGE"));
+ }
+
+ @Test
+ public void testDenseCompactTableWithoutRegulars() throws Throwable
+ {
+ String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
+ "pk1 varint," +
+ "ck1 int," +
+ "PRIMARY KEY (pk1, ck1))" +
+ " WITH COMPACT STORAGE");
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
+ "\tpk1 varint,\n" +
+ "\tck1 int,\n" +
+ "\tPRIMARY KEY (pk1, ck1))\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND COMPACT STORAGE"));
+ }
+
+ @Test
+ public void testCompactDynamic() throws Throwable
+ {
+ String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
+ "pk1 varint," +
+ "ck1 int," +
+ "reg int," +
+ "PRIMARY KEY (pk1, ck1))" +
+ " WITH COMPACT STORAGE");
+
+ ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
+ "\tpk1 varint,\n" +
+ "\tck1 int,\n" +
+ "\treg int,\n" +
+ "\tPRIMARY KEY (pk1, ck1))\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND COMPACT STORAGE"));
+ }
+
+ @Test
+ public void testDynamicComposite() throws Throwable
+ {
+ Map<Byte, AbstractType<?>> aliases = new HashMap<>();
+ aliases.put((byte)'a', BytesType.instance);
+ aliases.put((byte)'b', BytesType.instance);
+ aliases.put((byte)'c', BytesType.instance);
+
+ String DYNAMIC_COMPOSITE = "dynamic_composite";
+ AbstractType<?> dynamicComposite = DynamicCompositeType.getInstance(aliases);
+
+ SchemaLoader.createKeyspace(DYNAMIC_COMPOSITE,
+ KeyspaceParams.simple(1),
+ SchemaLoader.denseCFMD(DYNAMIC_COMPOSITE, DYNAMIC_COMPOSITE, dynamicComposite));
+
+ ColumnFamilyStore cfs = Keyspace.open(DYNAMIC_COMPOSITE).getColumnFamilyStore(DYNAMIC_COMPOSITE);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "CREATE TABLE IF NOT EXISTS " + DYNAMIC_COMPOSITE + "." + DYNAMIC_COMPOSITE + " (\n" +
+ "\tkey ascii,\n" +
+ "\tcols 'org.apache.cassandra.db.marshal.DynamicCompositeType(a=>org.apache.cassandra.db.marshal.BytesType,b=>org.apache.cassandra.db.marshal.BytesType,c=>org.apache.cassandra.db.marshal.BytesType)',\n" +
+ "\tval ascii,\n" +
+ "\tPRIMARY KEY (key, cols))\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND COMPACT STORAGE"));
+ }
+
+ @Test
+ public void superColumnFamilyTest() throws Throwable
+ {
+ final String KEYSPACE = "thrift_compact_table_with_supercolumns_test";
+ final String TABLE = "test_table_1";
+
+ CFMetaData cfm = CFMetaData.Builder.createSuper(KEYSPACE, TABLE, false)
+ .addPartitionKey("pk", BytesType.instance)
+ .addClusteringColumn("c1", AsciiType.instance)
+ .addClusteringColumn("c2", AsciiType.instance)
+ .addRegularColumn("", MapType.getInstance(Int32Type.instance, AsciiType.instance, true))
+ .build();
+
+ SchemaLoader.createKeyspace(KEYSPACE,
+ KeyspaceParams.simple(1),
+ cfm);
+
+ ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE);
+
+ assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+ "/*\n" +
+ "Warning: Table " + KEYSPACE + "." + TABLE + " omitted because it has constructs not compatible with CQL (was created via legacy API).\n\n" +
+ "Approximate structure, for reference:\n" +
+ "(this should not be used to reproduce this schema)\n\n" +
+ "CREATE TABLE IF NOT EXISTS " + KEYSPACE + "." + TABLE + " (\n" +
+ "\tpk blob,\n" +
+ "\tc1 ascii,\n" +
+ "\tc2 ascii,\n" +
+ "\t\"\" map<int, ascii>,\n" +
+ "\tPRIMARY KEY (pk, c1, c2))\n" +
+ "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+ "\tAND COMPACT STORAGE"));
+ }
+}
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index af43152..5d01886 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -43,6 +43,7 @@
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.metrics.ClearableHistogram;
import org.apache.cassandra.schema.KeyspaceParams;
@@ -350,8 +351,9 @@
for (int version = 1; version <= 2; ++version)
{
- Descriptor existing = new Descriptor(cfs.getDirectories().getDirectoryForNewSSTables(), KEYSPACE2, CF_STANDARD1, version);
- Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), KEYSPACE2, CF_STANDARD1, version);
+ Descriptor existing = new Descriptor(cfs.getDirectories().getDirectoryForNewSSTables(), KEYSPACE2, CF_STANDARD1, version,
+ SSTableFormat.Type.BIG);
+ Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), KEYSPACE2, CF_STANDARD1, version, SSTableFormat.Type.BIG);
for (Component c : new Component[]{ Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.STATS })
assertTrue("Cannot find backed-up file:" + desc.filenameFor(c), new File(desc.filenameFor(c)).exists());
}
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index b09bfad..a043b1b 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -31,7 +31,6 @@
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.db.rows.BufferCell;
import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.CellPath;
import org.apache.cassandra.db.rows.Cells;
import org.apache.cassandra.db.context.CounterContext;
import org.apache.cassandra.exceptions.ConfigurationException;
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index f8d01a8..9a1c0bd 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -41,6 +41,7 @@
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.schema.IndexMetadata;
import org.apache.cassandra.service.DefaultFSErrorHandler;
@@ -117,7 +118,7 @@
private static void createFakeSSTable(File dir, String cf, int gen, List<File> addTo) throws IOException
{
- Descriptor desc = new Descriptor(dir, KS, cf, gen);
+ Descriptor desc = new Descriptor(dir, KS, cf, gen, SSTableFormat.Type.BIG);
for (Component c : new Component[]{ Component.DATA, Component.PRIMARY_INDEX, Component.FILTER })
{
File f = new File(desc.filenameFor(c));
@@ -152,7 +153,7 @@
Directories directories = new Directories(cfm);
assertEquals(cfDir(cfm), directories.getDirectoryForNewSSTables());
- Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1);
+ Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1, SSTableFormat.Type.BIG);
File snapshotDir = new File(cfDir(cfm), File.separator + Directories.SNAPSHOT_SUBDIR + File.separator + "42");
assertEquals(snapshotDir, Directories.getSnapshotDirectory(desc, "42"));
@@ -186,8 +187,8 @@
{
assertEquals(cfDir(INDEX_CFM), dir);
}
- Descriptor parentDesc = new Descriptor(parentDirectories.getDirectoryForNewSSTables(), KS, PARENT_CFM.cfName, 0);
- Descriptor indexDesc = new Descriptor(indexDirectories.getDirectoryForNewSSTables(), KS, INDEX_CFM.cfName, 0);
+ Descriptor parentDesc = new Descriptor(parentDirectories.getDirectoryForNewSSTables(), KS, PARENT_CFM.cfName, 0, SSTableFormat.Type.BIG);
+ Descriptor indexDesc = new Descriptor(indexDirectories.getDirectoryForNewSSTables(), KS, INDEX_CFM.cfName, 0, SSTableFormat.Type.BIG);
// snapshot dir should be created under its parent's
File parentSnapshotDirectory = Directories.getSnapshotDirectory(parentDesc, "test");
@@ -204,9 +205,9 @@
indexDirectories.snapshotCreationTime("test"));
// check true snapshot size
- Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.cfName, 0);
+ Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.cfName, 0, SSTableFormat.Type.BIG);
createFile(parentSnapshot.filenameFor(Component.DATA), 30);
- Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.cfName, 0);
+ Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.cfName, 0, SSTableFormat.Type.BIG);
createFile(indexSnapshot.filenameFor(Component.DATA), 40);
assertEquals(30, parentDirectories.trueSnapshotsSize());
@@ -354,7 +355,7 @@
final String n = Long.toString(System.nanoTime());
Callable<File> directoryGetter = new Callable<File>() {
public File call() throws Exception {
- Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1);
+ Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1, SSTableFormat.Type.BIG);
return Directories.getSnapshotDirectory(desc, n);
}
};
diff --git a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
index 73f97fa..6c8eed5 100644
--- a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
@@ -52,7 +52,6 @@
import org.apache.cassandra.db.rows.UnfilteredSerializer;
import org.apache.cassandra.dht.Murmur3Partitioner;
import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.compress.BufferType;
import org.apache.cassandra.io.sstable.IndexInfo;
import org.apache.cassandra.io.sstable.format.SSTableFlushObserver;
import org.apache.cassandra.io.sstable.format.Version;
@@ -803,7 +802,7 @@
RowIndexEntry rie = new RowIndexEntry(0L)
{
- public IndexInfoRetriever openWithIndex(SegmentedFile indexFile)
+ public IndexInfoRetriever openWithIndex(FileHandle indexFile)
{
return new IndexInfoRetriever()
{
diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java
index 3b0293c..b0cd4fc 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -21,12 +21,9 @@
import org.junit.Test;
import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
import org.apache.cassandra.Util;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java
index d216860..53bf655 100644
--- a/test/unit/org/apache/cassandra/db/VerifyTest.java
+++ b/test/unit/org/apache/cassandra/db/VerifyTest.java
@@ -31,7 +31,6 @@
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.exceptions.WriteTimeoutException;
import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.FileUtils;
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
index e308a2f..68ce57d 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
@@ -51,8 +51,7 @@
@Before
public void before() throws IOException
{
- // disable reserve segment to get more deterministic allocation/testing of CDC boundary states
- CommitLog.instance.forceRecycleAllSegments();
+ CommitLog.instance.resetUnsafe(true);
for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).listFiles())
FileUtils.deleteWithConfirm(f);
}
@@ -120,7 +119,7 @@
for (int i = 0; i < 8; i++)
{
new RowUpdateBuilder(currentTableMetadata(), 0, i)
- .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3))
+ .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 4)) // fit 3 in a segment
.build().apply();
}
@@ -136,7 +135,7 @@
for (int i = 0; i < 8; i++)
{
new RowUpdateBuilder(currentTableMetadata(), 0, i)
- .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3))
+ .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 4))
.build().apply();
}
// 4 total again, 3 CONTAINS, 1 in waiting PERMITTED
@@ -215,6 +214,6 @@
private void expectCurrentCDCState(CDCState state)
{
Assert.assertEquals("Received unexpected CDCState on current allocatingFrom segment.",
- state, CommitLog.instance.segmentManager.allocatingFrom.getCDCState());
+ state, CommitLog.instance.segmentManager.allocatingFrom().getCDCState());
}
}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
index b777389..397a8eb 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
@@ -21,6 +21,7 @@
package org.apache.cassandra.db.commitlog;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
import java.util.Random;
import java.util.concurrent.Semaphore;
import javax.naming.ConfigurationException;
@@ -51,6 +52,7 @@
public class CommitLogSegmentManagerTest
{
//Block commit log service from syncing
+ @SuppressWarnings("unused")
private static final Semaphore allowSync = new Semaphore(0);
private static final String KEYSPACE1 = "CommitLogTest";
@@ -66,6 +68,7 @@
DatabaseDescriptor.setCommitLogSegmentSize(1);
DatabaseDescriptor.setCommitLogSync(CommitLogSync.periodic);
DatabaseDescriptor.setCommitLogSyncPeriod(10 * 1000);
+ DatabaseDescriptor.setCommitLogMaxCompressionBuffersPerPool(3);
SchemaLoader.prepareServer();
SchemaLoader.createKeyspace(KEYSPACE1,
KeyspaceParams.simple(1),
@@ -109,11 +112,14 @@
}
Thread.sleep(1000);
- // Should only be able to create 3 segments (not 7) because it blocks waiting for truncation that never comes.
+ // Should only be able to create 3 segments not 7 because it blocks waiting for truncation that never comes
Assert.assertEquals(3, clsm.getActiveSegments().size());
- clsm.getActiveSegments().forEach(segment -> clsm.recycleSegment(segment));
+ // Discard the currently active segments so allocation can continue.
+ // Take snapshot of the list, otherwise this will also discard newly allocated segments.
+ new ArrayList<>(clsm.getActiveSegments()).forEach( clsm::archiveAndDiscard );
+ // The allocated count should reach the limit again.
Util.spinAssertEquals(3, () -> clsm.getActiveSegments().size(), 5);
}
}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 23ec58b..4bc5f6b 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -277,7 +277,7 @@
// "Flush": this won't delete anything
UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
- CommitLog.instance.sync(true);
+ CommitLog.instance.sync();
CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getCurrentPosition());
assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
@@ -509,19 +509,6 @@
runExpecting(() -> testRecovery(new CommitLogDescriptor(4, commitLogCompression, encryptionContext), logData), expected);
}
- protected void testRecovery(byte[] logData) throws Exception
- {
- Pair<File, Integer> pair = tmpFile();
- try (RandomAccessFile raf = new RandomAccessFile(pair.left, "rw"))
- {
- raf.seek(pair.right);
- raf.write(logData);
- raf.close();
-
- CommitLog.instance.recoverFiles(pair.left); //CASSANDRA-1119 / CASSANDRA-1179 throw on failure*/
- }
- }
-
@Test
public void testTruncateWithoutSnapshot() throws ExecutionException, InterruptedException, IOException
{
@@ -607,7 +594,7 @@
cellCount += 1;
CommitLog.instance.add(rm2);
- CommitLog.instance.sync(true);
+ CommitLog.instance.sync();
SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, CommitLogPosition.NONE, cfs.metadata);
List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
@@ -644,7 +631,7 @@
}
}
- CommitLog.instance.sync(true);
+ CommitLog.instance.sync();
SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, commitLogPosition, cfs.metadata);
List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
@@ -674,6 +661,10 @@
@Override
public void handleMutation(Mutation m, int size, int entryLocation, CommitLogDescriptor desc)
{
+ // Filter out system writes that could flake the test.
+ if (!KEYSPACE1.equals(m.getKeyspaceName()))
+ return;
+
if (entryLocation <= filterPosition.position)
{
// Skip over this mutation.
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
index 9a22b04..7f43378 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
@@ -40,7 +40,7 @@
public CommitLogTestReplayer(Predicate<Mutation> processor) throws IOException
{
super(CommitLog.instance, CommitLogPosition.NONE, null, ReplayFilter.create());
- CommitLog.instance.sync(true);
+ CommitLog.instance.sync();
this.processor = processor;
commitLogReader = new CommitLogTestReader();
diff --git a/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java
index 88300a1..1e8d944 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.db.commitlog;
-import java.io.DataInput;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
diff --git a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
index 37f1731..1fae80d 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
@@ -31,7 +31,6 @@
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.RowUpdateBuilder;
import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
-import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
import org.apache.cassandra.schema.KeyspaceParams;
import org.apache.cassandra.service.GCInspector;
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index b264553..1a04e1f 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -33,7 +33,6 @@
import org.apache.cassandra.Util;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.BTreeRow;
import org.apache.cassandra.db.filter.ColumnFilter;
import org.apache.cassandra.db.lifecycle.SSTableSet;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
diff --git a/test/unit/org/apache/cassandra/db/filter/SliceTest.java b/test/unit/org/apache/cassandra/db/filter/SliceTest.java
index b0705ce..c2589d8 100644
--- a/test/unit/org/apache/cassandra/db/filter/SliceTest.java
+++ b/test/unit/org/apache/cassandra/db/filter/SliceTest.java
@@ -19,12 +19,10 @@
package org.apache.cassandra.db.filter;
-import java.awt.*;
import java.nio.ByteBuffer;
import java.util.*;
import java.util.List;
-import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.db.*;
import org.junit.Test;
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
index 5ed22e4..2620a31 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@ -46,15 +46,13 @@
import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.compaction.*;
import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.sstable.metadata.MetadataType;
import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.BufferedSegmentedFile;
-import org.apache.cassandra.io.util.ChannelProxy;
import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.io.util.FileHandle;
import org.apache.cassandra.utils.AlwaysPresentFilter;
import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
import org.apache.cassandra.utils.concurrent.Transactional;
@@ -1156,7 +1154,7 @@
private static SSTableReader sstable(File dataFolder, ColumnFamilyStore cfs, int generation, int size) throws IOException
{
- Descriptor descriptor = new Descriptor(dataFolder, cfs.keyspace.getName(), cfs.getTableName(), generation);
+ Descriptor descriptor = new Descriptor(dataFolder, cfs.keyspace.getName(), cfs.getTableName(), generation, SSTableFormat.Type.BIG);
Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
for (Component component : components)
{
@@ -1169,8 +1167,8 @@
}
}
- SegmentedFile dFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.DATA))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
- SegmentedFile iFile = new BufferedSegmentedFile(new ChannelProxy(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))), RandomAccessReader.DEFAULT_BUFFER_SIZE, 0);
+ FileHandle dFile = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).complete();
+ FileHandle iFile = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX)).complete();
SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList());
StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
diff --git a/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
index 0054163..b6160bf 100644
--- a/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
@@ -19,10 +19,7 @@
package org.apache.cassandra.db.marshal;
import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Random;
import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
import junit.framework.Assert;
import org.apache.cassandra.serializers.MarshalException;
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index ad07165..4c5cc7c 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -25,18 +25,13 @@
import java.util.UUID;
import com.google.common.collect.ImmutableMap;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Before;
-import org.junit.BeforeClass;
import org.junit.Test;
-import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.Util;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.RandomPartitioner;
import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.locator.TokenMetadata;
import org.apache.cassandra.service.StorageService;
diff --git a/test/unit/org/apache/cassandra/index/StubIndex.java b/test/unit/org/apache/cassandra/index/StubIndex.java
index 5f9d1f3..0b7b32f 100644
--- a/test/unit/org/apache/cassandra/index/StubIndex.java
+++ b/test/unit/org/apache/cassandra/index/StubIndex.java
@@ -31,7 +31,6 @@
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.db.partitions.PartitionIterator;
import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
import org.apache.cassandra.db.rows.Row;
import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.index.transactions.IndexTransaction;
diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index 46d1a3c..47e3895 100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@ -46,7 +46,6 @@
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.SyntaxException;
import org.apache.cassandra.index.sasi.conf.ColumnIndex;
import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
import org.apache.cassandra.index.sasi.exceptions.TimeQuotaExceededException;
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 309083b..2de6b62 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -70,39 +70,38 @@
{
File f = File.createTempFile("compressed6791_", "3");
String filename = f.getAbsolutePath();
- try(ChannelProxy channel = new ChannelProxy(f))
+ MetadataCollector sstableMetadataCollector = new MetadataCollector(new ClusteringComparator(BytesType.instance));
+ try(CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata",
+ null, SequentialWriterOption.DEFAULT,
+ CompressionParams.snappy(32),
+ sstableMetadataCollector))
{
- MetadataCollector sstableMetadataCollector = new MetadataCollector(new ClusteringComparator(BytesType.instance));
- try(CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata",
- null, SequentialWriterOption.DEFAULT,
- CompressionParams.snappy(32),
- sstableMetadataCollector))
- {
+ for (int i = 0; i < 20; i++)
+ writer.write("x".getBytes());
- for (int i = 0; i < 20; i++)
- writer.write("x".getBytes());
+ DataPosition mark = writer.mark();
+ // write enough garbage to create new chunks:
+ for (int i = 0; i < 40; ++i)
+ writer.write("y".getBytes());
- DataPosition mark = writer.mark();
- // write enough garbage to create new chunks:
- for (int i = 0; i < 40; ++i)
- writer.write("y".getBytes());
+ writer.resetAndTruncate(mark);
- writer.resetAndTruncate(mark);
+ for (int i = 0; i < 20; i++)
+ writer.write("x".getBytes());
+ writer.finish();
+ }
- for (int i = 0; i < 20; i++)
- writer.write("x".getBytes());
- writer.finish();
- }
-
- try(RandomAccessReader reader = RandomAccessReader.builder(channel)
- .compression(new CompressionMetadata(filename + ".metadata", f.length(), ChecksumType.CRC32))
- .build())
- {
- String res = reader.readLine();
- assertEquals(res, "xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx");
- assertEquals(40, res.length());
- }
+ try (FileHandle.Builder builder = new FileHandle.Builder(filename)
+ .withCompressionMetadata(new CompressionMetadata(filename + ".metadata",
+ f.length(),
+ ChecksumType.CRC32));
+ FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
+ {
+ String res = reader.readLine();
+ assertEquals(res, "xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx");
+ assertEquals(40, res.length());
}
finally
{
@@ -117,56 +116,39 @@
private static void testResetAndTruncate(File f, boolean compressed, boolean usemmap, int junkSize) throws IOException
{
final String filename = f.getAbsolutePath();
- try(ChannelProxy channel = new ChannelProxy(f))
- {
- MetadataCollector sstableMetadataCollector = new MetadataCollector(new ClusteringComparator(BytesType.instance));
- try(SequentialWriter writer = compressed
+ MetadataCollector sstableMetadataCollector = new MetadataCollector(new ClusteringComparator(BytesType.instance));
+ try(SequentialWriter writer = compressed
? new CompressedSequentialWriter(f, filename + ".metadata",
- null, SequentialWriterOption.DEFAULT,
- CompressionParams.snappy(), sstableMetadataCollector)
+ null, SequentialWriterOption.DEFAULT,
+ CompressionParams.snappy(), sstableMetadataCollector)
: new SequentialWriter(f))
+ {
+ writer.write("The quick ".getBytes());
+ DataPosition mark = writer.mark();
+ writer.write("blue fox jumps over the lazy dog".getBytes());
+
+ // write enough to be sure to change chunk
+ for (int i = 0; i < junkSize; ++i)
{
- writer.write("The quick ".getBytes());
- DataPosition mark = writer.mark();
- writer.write("blue fox jumps over the lazy dog".getBytes());
-
- // write enough to be sure to change chunk
- for (int i = 0; i < junkSize; ++i)
- {
- writer.write((byte) 1);
- }
-
- writer.resetAndTruncate(mark);
- writer.write("brown fox jumps over the lazy dog".getBytes());
- writer.finish();
- }
- assert f.exists();
-
- CompressionMetadata compressionMetadata = compressed ? new CompressionMetadata(filename + ".metadata", f.length(), ChecksumType.CRC32) : null;
- RandomAccessReader.Builder builder = RandomAccessReader.builder(channel);
- if (compressed)
- builder.compression(compressionMetadata);
-
- MmappedRegions regions = null;
- if (usemmap)
- {
- regions = compressed
- ? MmappedRegions.map(channel, compressionMetadata)
- : MmappedRegions.map(channel, f.length());
- builder.regions(regions);
+ writer.write((byte) 1);
}
- try(RandomAccessReader reader = builder.build())
- {
- String expected = "The quick brown fox jumps over the lazy dog";
- assertEquals(expected.length(), reader.length());
- byte[] b = new byte[expected.length()];
- reader.readFully(b);
- assert new String(b).equals(expected) : "Expecting '" + expected + "', got '" + new String(b) + '\'';
- }
+ writer.resetAndTruncate(mark);
+ writer.write("brown fox jumps over the lazy dog".getBytes());
+ writer.finish();
+ }
+ assert f.exists();
- if (regions != null)
- regions.close();
+ CompressionMetadata compressionMetadata = compressed ? new CompressionMetadata(filename + ".metadata", f.length(), ChecksumType.CRC32) : null;
+ try (FileHandle.Builder builder = new FileHandle.Builder(filename).mmapped(usemmap).withCompressionMetadata(compressionMetadata);
+ FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
+ {
+ String expected = "The quick brown fox jumps over the lazy dog";
+ assertEquals(expected.length(), reader.length());
+ byte[] b = new byte[expected.length()];
+ reader.readFully(b);
+ assert new String(b).equals(expected) : "Expecting '" + expected + "', got '" + new String(b) + '\'';
}
finally
{
@@ -201,70 +183,69 @@
writer.finish();
}
- try(ChannelProxy channel = new ChannelProxy(file))
- {
- // open compression metadata and get chunk information
- CompressionMetadata meta = new CompressionMetadata(metadata.getPath(), file.length(), ChecksumType.CRC32);
- CompressionMetadata.Chunk chunk = meta.chunkFor(0);
+ // open compression metadata and get chunk information
+ CompressionMetadata meta = new CompressionMetadata(metadata.getPath(), file.length(), ChecksumType.CRC32);
+ CompressionMetadata.Chunk chunk = meta.chunkFor(0);
- try(RandomAccessReader reader = RandomAccessReader.builder(channel).compression(meta).build())
- {// read and verify compressed data
- assertEquals(CONTENT, reader.readLine());
+ try (FileHandle.Builder builder = new FileHandle.Builder(file.getPath()).withCompressionMetadata(meta);
+ FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
+ {// read and verify compressed data
+ assertEquals(CONTENT, reader.readLine());
- Random random = new Random();
- RandomAccessFile checksumModifier = null;
+ Random random = new Random();
+ RandomAccessFile checksumModifier = null;
- try
+ try
+ {
+ checksumModifier = new RandomAccessFile(file, "rw");
+ byte[] checksum = new byte[4];
+
+ // seek to the end of the compressed chunk
+ checksumModifier.seek(chunk.length);
+ // read checksum bytes
+ checksumModifier.read(checksum);
+ // seek back to the chunk end
+ checksumModifier.seek(chunk.length);
+
+ // lets modify one byte of the checksum on each iteration
+ for (int i = 0; i < checksum.length; i++)
{
- checksumModifier = new RandomAccessFile(file, "rw");
- byte[] checksum = new byte[4];
+ checksumModifier.write(random.nextInt());
+ SyncUtil.sync(checksumModifier); // making sure that change was synced with disk
- // seek to the end of the compressed chunk
- checksumModifier.seek(chunk.length);
- // read checksum bytes
- checksumModifier.read(checksum);
- // seek back to the chunk end
- checksumModifier.seek(chunk.length);
-
- // lets modify one byte of the checksum on each iteration
- for (int i = 0; i < checksum.length; i++)
+ try (final RandomAccessReader r = fh.createReader())
{
- checksumModifier.write(random.nextInt());
- SyncUtil.sync(checksumModifier); // making sure that change was synced with disk
-
- try (final RandomAccessReader r = RandomAccessReader.builder(channel).compression(meta).build())
+ Throwable exception = null;
+ try
{
- Throwable exception = null;
- try
- {
- r.readLine();
- }
- catch (Throwable t)
- {
- exception = t;
- }
- assertNotNull(exception);
- assertSame(exception.getClass(), CorruptSSTableException.class);
- assertSame(exception.getCause().getClass(), CorruptBlockException.class);
+ r.readLine();
}
- }
-
- // lets write original checksum and check if we can read data
- updateChecksum(checksumModifier, chunk.length, checksum);
-
- try (RandomAccessReader cr = RandomAccessReader.builder(channel).compression(meta).build())
- {
- // read and verify compressed data
- assertEquals(CONTENT, cr.readLine());
- // close reader
+ catch (Throwable t)
+ {
+ exception = t;
+ }
+ assertNotNull(exception);
+ assertSame(exception.getClass(), CorruptSSTableException.class);
+ assertSame(exception.getCause().getClass(), CorruptBlockException.class);
}
}
- finally
+
+ // lets write original checksum and check if we can read data
+ updateChecksum(checksumModifier, chunk.length, checksum);
+
+ try (RandomAccessReader cr = fh.createReader())
{
- if (checksumModifier != null)
- checksumModifier.close();
+ // read and verify compressed data
+ assertEquals(CONTENT, cr.readLine());
+ // close reader
}
}
+ finally
+ {
+ if (checksumModifier != null)
+ checksumModifier.close();
+ }
}
}
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
index 9959c7b..fa9643b 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -34,7 +34,6 @@
import junit.framework.Assert;
import org.apache.cassandra.db.ClusteringComparator;
-import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.BytesType;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -82,43 +81,42 @@
private void testWrite(File f, int bytesToTest) throws IOException
{
final String filename = f.getAbsolutePath();
- final ChannelProxy channel = new ChannelProxy(f);
+ MetadataCollector sstableMetadataCollector = new MetadataCollector(new ClusteringComparator(Collections.singletonList(BytesType.instance)));
- try
+ byte[] dataPre = new byte[bytesToTest];
+ byte[] rawPost = new byte[bytesToTest];
+ try (CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata",
+ null, SequentialWriterOption.DEFAULT,
+ compressionParameters,
+ sstableMetadataCollector))
{
- MetadataCollector sstableMetadataCollector = new MetadataCollector(new ClusteringComparator(Arrays.<AbstractType<?>>asList(BytesType.instance)));
+ Random r = new Random(42);
- byte[] dataPre = new byte[bytesToTest];
- byte[] rawPost = new byte[bytesToTest];
- try (CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata",
- null, SequentialWriterOption.DEFAULT,
- compressionParameters,
- sstableMetadataCollector))
+ // Test both write with byte[] and ByteBuffer
+ r.nextBytes(dataPre);
+ r.nextBytes(rawPost);
+ ByteBuffer dataPost = makeBB(bytesToTest);
+ dataPost.put(rawPost);
+ dataPost.flip();
+
+ writer.write(dataPre);
+ DataPosition mark = writer.mark();
+
+ // Write enough garbage to transition chunk
+ for (int i = 0; i < CompressionParams.DEFAULT_CHUNK_LENGTH; i++)
{
- Random r = new Random(42);
-
- // Test both write with byte[] and ByteBuffer
- r.nextBytes(dataPre);
- r.nextBytes(rawPost);
- ByteBuffer dataPost = makeBB(bytesToTest);
- dataPost.put(rawPost);
- dataPost.flip();
-
- writer.write(dataPre);
- DataPosition mark = writer.mark();
-
- // Write enough garbage to transition chunk
- for (int i = 0; i < CompressionParams.DEFAULT_CHUNK_LENGTH; i++)
- {
- writer.write((byte)i);
- }
- writer.resetAndTruncate(mark);
- writer.write(dataPost);
- writer.finish();
+ writer.write((byte)i);
}
+ writer.resetAndTruncate(mark);
+ writer.write(dataPost);
+ writer.finish();
+ }
- assert f.exists();
- RandomAccessReader reader = RandomAccessReader.builder(channel).compression(new CompressionMetadata(filename + ".metadata", f.length(), ChecksumType.CRC32)).build();
+ assert f.exists();
+ try (FileHandle.Builder builder = new FileHandle.Builder(filename).withCompressionMetadata(new CompressionMetadata(filename + ".metadata", f.length(), ChecksumType.CRC32));
+ FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
+ {
assertEquals(dataPre.length + rawPost.length, reader.length());
byte[] result = new byte[(int)reader.length()];
@@ -134,9 +132,6 @@
}
finally
{
- // cleanup
- channel.close();
-
if (f.exists())
f.delete();
File metadata = new File(f + ".metadata");
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressorTest.java b/test/unit/org/apache/cassandra/io/compress/CompressorTest.java
index 1e24d03..617e04e 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressorTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressorTest.java
@@ -33,7 +33,6 @@
import org.apache.cassandra.io.util.RandomAccessReader;
import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.memory.BufferPool;
public class CompressorTest
{
diff --git a/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
index f769293..64367dc 100644
--- a/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
@@ -27,7 +27,6 @@
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.io.sstable.format.SSTableFormat;
-import org.apache.cassandra.io.sstable.format.big.BigFormat;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.Pair;
@@ -77,14 +76,14 @@
private void testFromFilenameFor(File dir)
{
// normal
- checkFromFilename(new Descriptor(dir, ksname, cfname, 1), false);
+ checkFromFilename(new Descriptor(dir, ksname, cfname, 1, SSTableFormat.Type.BIG), false);
// skip component (for streaming lock file)
- checkFromFilename(new Descriptor(dir, ksname, cfname, 2), true);
+ checkFromFilename(new Descriptor(dir, ksname, cfname, 2, SSTableFormat.Type.BIG), true);
// secondary index
String idxName = "myidx";
File idxDir = new File(dir.getAbsolutePath() + File.separator + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName);
- checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 4), false);
+ checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 4, SSTableFormat.Type.BIG), false);
// legacy version
checkFromFilename(new Descriptor("ja", dir, ksname, cfname, 1, SSTableFormat.Type.LEGACY), false);
diff --git a/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java b/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
index f97356a..5536efd 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
@@ -25,7 +25,6 @@
import org.junit.runner.RunWith;
import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.cql3.CQLTester;
import org.apache.cassandra.metrics.CacheMetrics;
import org.apache.cassandra.service.CacheService;
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 803e275..cc2011f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -27,7 +27,6 @@
import org.junit.After;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
@@ -91,6 +90,12 @@
@BeforeClass
public static void defineSchema() throws ConfigurationException
{
+ String scp = System.getProperty(LEGACY_SSTABLE_PROP);
+ Assert.assertNotNull("System property " + LEGACY_SSTABLE_ROOT + " not set", scp);
+
+ LEGACY_SSTABLE_ROOT = new File(scp).getAbsoluteFile();
+ Assert.assertTrue("System property " + LEGACY_SSTABLE_ROOT + " does not specify a directory", LEGACY_SSTABLE_ROOT.isDirectory());
+
SchemaLoader.prepareServer();
StorageService.instance.initServer();
Keyspace.setInitialized();
@@ -99,10 +104,7 @@
{
createTables(legacyVersion);
}
- String scp = System.getProperty(LEGACY_SSTABLE_PROP);
- assert scp != null;
- LEGACY_SSTABLE_ROOT = new File(scp).getAbsoluteFile();
- assert LEGACY_SSTABLE_ROOT.isDirectory();
+
}
@After
@@ -455,7 +457,9 @@
private static void copySstablesToTestData(String legacyVersion, String table, File cfDir) throws IOException
{
- for (File file : getTableDir(legacyVersion, table).listFiles())
+ File tableDir = getTableDir(legacyVersion, table);
+ Assert.assertTrue("The table directory " + tableDir + " was not found", tableDir.isDirectory());
+ for (File file : tableDir.listFiles())
{
copyFile(cfDir, file);
}
@@ -473,10 +477,11 @@
{
File target = new File(cfDir, file.getName());
int rd;
- FileInputStream is = new FileInputStream(file);
- FileOutputStream os = new FileOutputStream(target);
- while ((rd = is.read(buf)) >= 0)
- os.write(buf, 0, rd);
+ try (FileInputStream is = new FileInputStream(file);
+ FileOutputStream os = new FileOutputStream(target);) {
+ while ((rd = is.read(buf)) >= 0)
+ os.write(buf, 0, rd);
+ }
}
}
}
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 151a995..0928ad4 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -31,7 +31,6 @@
import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.Util;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.cql3.Operator;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.compaction.CompactionManager;
@@ -47,7 +46,6 @@
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.FileDataInput;
import org.apache.cassandra.io.util.MmappedRegions;
-import org.apache.cassandra.io.util.SegmentedFile;
import org.apache.cassandra.schema.CachingParams;
import org.apache.cassandra.schema.KeyspaceParams;
import org.apache.cassandra.service.CacheService;
@@ -393,13 +391,7 @@
SSTableReader sstable = indexCfs.getLiveSSTables().iterator().next();
assert sstable.first.getToken() instanceof LocalToken;
- try (SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(),
- false);
- SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(),
- sstable.compression))
- {
- sstable.saveSummary(ibuilder, dbuilder);
- }
+ sstable.saveSummary();
SSTableReader reopened = SSTableReader.open(sstable.descriptor);
assert reopened.first.getToken() instanceof LocalToken;
reopened.selfRef().release();
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index 5c7ff02..df9d1aa 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -28,6 +28,7 @@
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.rows.*;
import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.service.ActiveRepairService;
@@ -76,7 +77,7 @@
File cfDir = new File(tempdir, keyspaceName + File.separator + cfname);
cfDir.mkdirs();
cfDir.deleteOnExit();
- File datafile = new File(new Descriptor(cfDir, keyspaceName, cfname, generation).filenameFor(Component.DATA));
+ File datafile = new File(new Descriptor(cfDir, keyspaceName, cfname, generation, SSTableFormat.Type.BIG).filenameFor(Component.DATA));
if (!datafile.createNewFile())
throw new IOException("unable to create file " + datafile);
datafile.deleteOnExit();
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
index 9c3bb19..68bc3c1 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
@@ -31,7 +31,6 @@
import org.junit.BeforeClass;
import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index a3382c4..62d0479 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -19,7 +19,6 @@
package org.apache.cassandra.io.sstable.metadata;
import java.io.File;
-import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.Collections;
@@ -30,12 +29,13 @@
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.SerializationHeader;
import org.apache.cassandra.db.commitlog.CommitLogPosition;
import org.apache.cassandra.dht.RandomPartitioner;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.sstable.format.Version;
import org.apache.cassandra.io.sstable.format.big.BigFormat;
import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
@@ -54,7 +54,7 @@
MetadataSerializer serializer = new MetadataSerializer();
File statsFile = serialize(originalMetadata, serializer, BigFormat.latestVersion);
- Descriptor desc = new Descriptor( statsFile.getParentFile(), "", "", 0);
+ Descriptor desc = new Descriptor(statsFile.getParentFile(), "", "", 0, SSTableFormat.Type.BIG);
try (RandomAccessReader in = RandomAccessReader.open(statsFile))
{
Map<MetadataType, MetadataComponent> deserialized = serializer.deserialize(desc, in, EnumSet.allOf(MetadataType.class));
@@ -67,7 +67,7 @@
}
public File serialize(Map<MetadataType, MetadataComponent> metadata, MetadataSerializer serializer, Version version)
- throws IOException, FileNotFoundException
+ throws IOException
{
// Serialize to tmp file
File statsFile = File.createTempFile(Component.STATS.name, null);
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index 8cdd4ea..8dcf708 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -18,9 +18,6 @@
*
*/
package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.SyncUtil;
import java.io.File;
import java.io.FileOutputStream;
@@ -28,12 +25,15 @@
import java.nio.ByteBuffer;
import java.util.Arrays;
+import org.junit.Test;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.SyncUtil;
+
import static org.apache.cassandra.Util.expectEOF;
import static org.apache.cassandra.Util.expectException;
-import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
+import static org.junit.Assert.assertTrue;
public class BufferedRandomAccessFileTest
{
@@ -41,9 +41,8 @@
public void testReadAndWrite() throws Exception
{
SequentialWriter w = createTempFile("braf");
- ChannelProxy channel = new ChannelProxy(w.getPath());
- // writting string of data to the file
+ // writing string of data to the file
byte[] data = "Hello".getBytes();
w.write(data);
assertEquals(data.length, w.length());
@@ -52,79 +51,85 @@
w.sync();
// reading small amount of data from file, this is handled by initial buffer
- RandomAccessReader r = RandomAccessReader.open(channel);
-
- byte[] buffer = new byte[data.length];
- assertEquals(data.length, r.read(buffer));
- assertTrue(Arrays.equals(buffer, data)); // we read exactly what we wrote
- assertEquals(r.read(), -1); // nothing more to read EOF
- assert r.bytesRemaining() == 0 && r.isEOF();
-
- r.close();
-
- // writing buffer bigger than page size, which will trigger reBuffer()
- byte[] bigData = new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE + 10];
-
- for (int i = 0; i < bigData.length; i++)
- bigData[i] = 'd';
-
- long initialPosition = w.position();
- w.write(bigData); // writing data
- assertEquals(w.position(), initialPosition + bigData.length);
- assertEquals(w.length(), initialPosition + bigData.length); // file size should equals to last position
-
- w.sync();
-
- r = RandomAccessReader.open(channel); // re-opening file in read-only mode
-
- // reading written buffer
- r.seek(initialPosition); // back to initial (before write) position
- data = new byte[bigData.length];
- long sizeRead = 0;
- for (int i = 0; i < data.length; i++)
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath()))
{
- data[i] = (byte) r.read();
- sizeRead++;
+ try (FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
+ {
+
+ byte[] buffer = new byte[data.length];
+ assertEquals(data.length, r.read(buffer));
+ assertTrue(Arrays.equals(buffer, data)); // we read exactly what we wrote
+ assertEquals(r.read(), -1); // nothing more to read EOF
+ assert r.bytesRemaining() == 0 && r.isEOF();
+ }
+
+ // writing buffer bigger than page size, which will trigger reBuffer()
+ byte[] bigData = new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE + 10];
+
+ for (int i = 0; i < bigData.length; i++)
+ bigData[i] = 'd';
+
+ long initialPosition = w.position();
+ w.write(bigData); // writing data
+ assertEquals(w.position(), initialPosition + bigData.length);
+ assertEquals(w.length(), initialPosition + bigData.length); // file size should equals to last position
+
+ w.sync();
+
+ // re-opening file in read-only mode
+ try (FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
+ {
+
+ // reading written buffer
+ r.seek(initialPosition); // back to initial (before write) position
+ data = new byte[bigData.length];
+ long sizeRead = 0;
+ for (int i = 0; i < data.length; i++)
+ {
+ data[i] = (byte) r.read();
+ sizeRead++;
+ }
+
+ assertEquals(sizeRead, data.length); // read exactly data.length bytes
+ assertEquals(r.getFilePointer(), initialPosition + data.length);
+ assertEquals(r.length(), initialPosition + bigData.length);
+ assertTrue(Arrays.equals(bigData, data));
+ assertTrue(r.bytesRemaining() == 0 && r.isEOF()); // we are at the of the file
+
+ // test readBytes(int) method
+ r.seek(0);
+ ByteBuffer fileContent = ByteBufferUtil.read(r, (int) w.length());
+ assertEquals(fileContent.limit(), w.length());
+ assert ByteBufferUtil.string(fileContent).equals("Hello" + new String(bigData));
+
+ // read the same buffer but using readFully(int)
+ data = new byte[bigData.length];
+ r.seek(initialPosition);
+ r.readFully(data);
+ assert r.bytesRemaining() == 0 && r.isEOF(); // we should be at EOF
+ assertTrue(Arrays.equals(bigData, data));
+
+ // try to read past mark (all methods should return -1)
+ data = new byte[10];
+ assertEquals(r.read(), -1);
+ assertEquals(r.read(data), -1);
+ assertEquals(r.read(data, 0, data.length), -1);
+
+ // test read(byte[], int, int)
+ r.seek(0);
+ data = new byte[20];
+ assertEquals(15, r.read(data, 0, 15));
+ assertTrue(new String(data).contains("Hellodddddddddd"));
+ for (int i = 16; i < data.length; i++)
+ {
+ assert data[i] == 0;
+ }
+
+ w.finish();
+ }
}
-
- assertEquals(sizeRead, data.length); // read exactly data.length bytes
- assertEquals(r.getFilePointer(), initialPosition + data.length);
- assertEquals(r.length(), initialPosition + bigData.length);
- assertTrue(Arrays.equals(bigData, data));
- assertTrue(r.bytesRemaining() == 0 && r.isEOF()); // we are at the of the file
-
- // test readBytes(int) method
- r.seek(0);
- ByteBuffer fileContent = ByteBufferUtil.read(r, (int) w.length());
- assertEquals(fileContent.limit(), w.length());
- assert ByteBufferUtil.string(fileContent).equals("Hello" + new String(bigData));
-
- // read the same buffer but using readFully(int)
- data = new byte[bigData.length];
- r.seek(initialPosition);
- r.readFully(data);
- assert r.bytesRemaining() == 0 && r.isEOF(); // we should be at EOF
- assertTrue(Arrays.equals(bigData, data));
-
- // try to read past mark (all methods should return -1)
- data = new byte[10];
- assertEquals(r.read(), -1);
- assertEquals(r.read(data), -1);
- assertEquals(r.read(data, 0, data.length), -1);
-
- // test read(byte[], int, int)
- r.seek(0);
- data = new byte[20];
- assertEquals(15, r.read(data, 0, 15));
- assertTrue(new String(data).contains("Hellodddddddddd"));
- for (int i = 16; i < data.length; i++)
- {
- assert data[i] == 0;
- }
-
- w.finish();
- r.close();
- channel.close();
}
@Test
@@ -137,20 +142,20 @@
byte[] in = generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE);
w.write(in);
- ChannelProxy channel = new ChannelProxy(w.getPath());
- RandomAccessReader r = RandomAccessReader.open(channel);
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
+ {
+ // Read it into a same size array.
+ byte[] out = new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE];
+ r.read(out);
- // Read it into a same size array.
- byte[] out = new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE];
- r.read(out);
+ // Cannot read any more.
+ int negone = r.read();
+ assert negone == -1 : "We read past the end of the file, should have gotten EOF -1. Instead, " + negone;
- // Cannot read any more.
- int negone = r.read();
- assert negone == -1 : "We read past the end of the file, should have gotten EOF -1. Instead, " + negone;
-
- r.close();
- w.finish();
- channel.close();
+ w.finish();
+ }
}
@Test
@@ -178,8 +183,9 @@
w.finish();
// will use cachedlength
- try (ChannelProxy channel = new ChannelProxy(tmpFile);
- RandomAccessReader r = RandomAccessReader.open(channel))
+ try (FileHandle.Builder builder = new FileHandle.Builder(tmpFile.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
{
assertEquals(lessThenBuffer.length + biggerThenBuffer.length, r.length());
}
@@ -200,26 +206,27 @@
w.write(data);
w.sync();
- final ChannelProxy channel = new ChannelProxy(w.getPath());
- final RandomAccessReader r = RandomAccessReader.open(channel);
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
+ {
- ByteBuffer content = ByteBufferUtil.read(r, (int) r.length());
+ ByteBuffer content = ByteBufferUtil.read(r, (int) r.length());
- // after reading whole file we should be at EOF
- assertEquals(0, ByteBufferUtil.compare(content, data));
- assert r.bytesRemaining() == 0 && r.isEOF();
+ // after reading whole file we should be at EOF
+ assertEquals(0, ByteBufferUtil.compare(content, data));
+ assert r.bytesRemaining() == 0 && r.isEOF();
- r.seek(0);
- content = ByteBufferUtil.read(r, 10); // reading first 10 bytes
- assertEquals(ByteBufferUtil.compare(content, "cccccccccc".getBytes()), 0);
- assertEquals(r.bytesRemaining(), r.length() - content.limit());
+ r.seek(0);
+ content = ByteBufferUtil.read(r, 10); // reading first 10 bytes
+ assertEquals(ByteBufferUtil.compare(content, "cccccccccc".getBytes()), 0);
+ assertEquals(r.bytesRemaining(), r.length() - content.limit());
- // trying to read more than file has right now
- expectEOF(() -> ByteBufferUtil.read(r, (int) r.length() + 10));
+ // trying to read more than file has right now
+ expectEOF(() -> ByteBufferUtil.read(r, (int) r.length() + 10));
- w.finish();
- r.close();
- channel.close();
+ w.finish();
+ }
}
@Test
@@ -230,24 +237,29 @@
w.write(data);
w.finish();
- final ChannelProxy channel = new ChannelProxy(w.getPath());
- final RandomAccessReader file = RandomAccessReader.open(channel);
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader file = fh.createReader())
+ {
+ file.seek(0);
+ assertEquals(file.getFilePointer(), 0);
+ assertEquals(file.bytesRemaining(), file.length());
- file.seek(0);
- assertEquals(file.getFilePointer(), 0);
- assertEquals(file.bytesRemaining(), file.length());
+ file.seek(20);
+ assertEquals(file.getFilePointer(), 20);
+ assertEquals(file.bytesRemaining(), file.length() - 20);
- file.seek(20);
- assertEquals(file.getFilePointer(), 20);
- assertEquals(file.bytesRemaining(), file.length() - 20);
+ // trying to seek past the end of the file should produce EOFException
+ expectException(() -> {
+ file.seek(file.length() + 30);
+ return null;
+ }, IllegalArgumentException.class);
- // trying to seek past the end of the file should produce EOFException
- expectException(() -> { file.seek(file.length() + 30); return null; }, IllegalArgumentException.class);
-
- expectException(() -> { file.seek(-1); return null; }, IllegalArgumentException.class); // throws IllegalArgumentException
-
- file.close();
- channel.close();
+ expectException(() -> {
+ file.seek(-1);
+ return null;
+ }, IllegalArgumentException.class); // throws IllegalArgumentException
+ }
}
@Test
@@ -257,28 +269,28 @@
w.write(generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE * 2));
w.finish();
- ChannelProxy channel = new ChannelProxy(w.getPath());
- RandomAccessReader file = RandomAccessReader.open(channel);
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader file = fh.createReader())
+ {
- file.seek(0); // back to the beginning of the file
- assertEquals(file.skipBytes(10), 10);
- assertEquals(file.bytesRemaining(), file.length() - 10);
+ file.seek(0); // back to the beginning of the file
+ assertEquals(file.skipBytes(10), 10);
+ assertEquals(file.bytesRemaining(), file.length() - 10);
- int initialPosition = (int) file.getFilePointer();
- // can't skip more than file size
- assertEquals(file.skipBytes((int) file.length() + 10), file.length() - initialPosition);
- assertEquals(file.getFilePointer(), file.length());
- assert file.bytesRemaining() == 0 && file.isEOF();
+ int initialPosition = (int) file.getFilePointer();
+ // can't skip more than file size
+ assertEquals(file.skipBytes((int) file.length() + 10), file.length() - initialPosition);
+ assertEquals(file.getFilePointer(), file.length());
+ assert file.bytesRemaining() == 0 && file.isEOF();
- file.seek(0);
+ file.seek(0);
- // skipping negative amount should return 0
- assertEquals(file.skipBytes(-1000), 0);
- assertEquals(file.getFilePointer(), 0);
- assertEquals(file.bytesRemaining(), file.length());
-
- file.close();
- channel.close();
+ // skipping negative amount should return 0
+ assertEquals(file.skipBytes(-1000), 0);
+ assertEquals(file.getFilePointer(), 0);
+ assertEquals(file.bytesRemaining(), file.length());
+ }
}
@Test
@@ -293,22 +305,23 @@
w.sync();
- ChannelProxy channel = new ChannelProxy(w.getPath());
- RandomAccessReader r = RandomAccessReader.open(channel);
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
+ {
- // position should change after skip bytes
- r.seek(0);
- r.skipBytes(15);
- assertEquals(r.getFilePointer(), 15);
+ // position should change after skip bytes
+ r.seek(0);
+ r.skipBytes(15);
+ assertEquals(r.getFilePointer(), 15);
- r.read();
- assertEquals(r.getFilePointer(), 16);
- r.read(new byte[4]);
- assertEquals(r.getFilePointer(), 20);
+ r.read();
+ assertEquals(r.getFilePointer(), 16);
+ r.read(new byte[4]);
+ assertEquals(r.getFilePointer(), 20);
- w.finish();
- r.close();
- channel.close();
+ w.finish();
+ }
}
@Test
@@ -330,10 +343,9 @@
for (final int offset : Arrays.asList(0, 8))
{
File file1 = writeTemporaryFile(new byte[16]);
- try (final ChannelProxy channel = new ChannelProxy(file1);
- final RandomAccessReader file = RandomAccessReader.builder(channel)
- .bufferSize(bufferSize)
- .build())
+ try (FileHandle.Builder builder = new FileHandle.Builder(file1.getPath()).bufferSize(bufferSize);
+ FileHandle fh = builder.complete();
+ RandomAccessReader file = fh.createReader())
{
expectEOF(() -> { file.readFully(target, offset, 17); return null; });
}
@@ -343,8 +355,9 @@
for (final int n : Arrays.asList(1, 2, 4, 8))
{
File file1 = writeTemporaryFile(new byte[16]);
- try (final ChannelProxy channel = new ChannelProxy(file1);
- final RandomAccessReader file = RandomAccessReader.builder(channel).bufferSize(bufferSize).build())
+ try (FileHandle.Builder builder = new FileHandle.Builder(file1.getPath()).bufferSize(bufferSize);
+ FileHandle fh = builder.complete();
+ RandomAccessReader file = fh.createReader())
{
expectEOF(() -> {
while (true)
@@ -375,24 +388,25 @@
w.sync();
- ChannelProxy channel = new ChannelProxy(w.getPath());
- RandomAccessReader r = RandomAccessReader.open(channel);
-
- assertEquals(r.bytesRemaining(), toWrite);
-
- for (int i = 1; i <= r.length(); i++)
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
{
- r.read();
- assertEquals(r.bytesRemaining(), r.length() - i);
+
+ assertEquals(r.bytesRemaining(), toWrite);
+
+ for (int i = 1; i <= r.length(); i++)
+ {
+ r.read();
+ assertEquals(r.bytesRemaining(), r.length() - i);
+ }
+
+ r.seek(0);
+ r.skipBytes(10);
+ assertEquals(r.bytesRemaining(), r.length() - 10);
+
+ w.finish();
}
-
- r.seek(0);
- r.skipBytes(10);
- assertEquals(r.bytesRemaining(), r.length() - 10);
-
- w.finish();
- r.close();
- channel.close();
}
@Test
@@ -402,7 +416,9 @@
tmpFile.deleteOnExit();
// Create the BRAF by filename instead of by file.
- try (final RandomAccessReader r = RandomAccessReader.open(new File(tmpFile.getPath())))
+ try (FileHandle.Builder builder = new FileHandle.Builder(tmpFile.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
{
assert tmpFile.getPath().equals(r.getPath());
@@ -454,33 +470,32 @@
w.finish();
- ChannelProxy channel = new ChannelProxy(w.getPath());
- RandomAccessReader file = RandomAccessReader.open(channel);
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader file = fh.createReader())
+ {
+ file.seek(10);
+ DataPosition mark = file.mark();
- file.seek(10);
- DataPosition mark = file.mark();
+ file.seek(file.length());
+ assertTrue(file.isEOF());
- file.seek(file.length());
- assertTrue(file.isEOF());
+ file.reset();
+ assertEquals(file.bytesRemaining(), 20);
- file.reset();
- assertEquals(file.bytesRemaining(), 20);
+ file.seek(file.length());
+ assertTrue(file.isEOF());
- file.seek(file.length());
- assertTrue(file.isEOF());
+ file.reset(mark);
+ assertEquals(file.bytesRemaining(), 20);
- file.reset(mark);
- assertEquals(file.bytesRemaining(), 20);
+ file.seek(file.length());
+ assertEquals(file.bytesPastMark(), 20);
+ assertEquals(file.bytesPastMark(mark), 20);
- file.seek(file.length());
- assertEquals(file.bytesPastMark(), 20);
- assertEquals(file.bytesPastMark(mark), 20);
-
- file.reset(mark);
- assertEquals(file.bytesPastMark(), 0);
-
- file.close();
- channel.close();
+ file.reset(mark);
+ assertEquals(file.bytesPastMark(), 0);
+ }
}
@Test(expected = AssertionError.class)
@@ -491,8 +506,9 @@
w.write(new byte[30]);
w.flush();
- try (ChannelProxy channel = new ChannelProxy(w.getPath());
- RandomAccessReader r = RandomAccessReader.open(channel))
+ try (FileHandle.Builder builder = new FileHandle.Builder(w.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader r = fh.createReader())
{
r.seek(10);
r.mark();
diff --git a/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java
index 0657f7f..72a08a1 100644
--- a/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/util/ChecksummedRandomAccessReaderTest.java
@@ -47,15 +47,15 @@
assert data.exists();
- RandomAccessReader reader = new ChecksummedRandomAccessReader.Builder(data, crc).build();
- byte[] b = new byte[expected.length];
- reader.readFully(b);
+ try (RandomAccessReader reader = ChecksummedRandomAccessReader.open(data, crc))
+ {
+ byte[] b = new byte[expected.length];
+ reader.readFully(b);
- assertArrayEquals(expected, b);
+ assertArrayEquals(expected, b);
- assertTrue(reader.isEOF());
-
- reader.close();
+ assertTrue(reader.isEOF());
+ }
}
@Test
@@ -75,24 +75,24 @@
assert data.exists();
- RandomAccessReader reader = new ChecksummedRandomAccessReader.Builder(data, crc).build();
+ try (RandomAccessReader reader = ChecksummedRandomAccessReader.open(data, crc))
+ {
- final int seekPosition = 66000;
- reader.seek(seekPosition);
+ final int seekPosition = 66000;
+ reader.seek(seekPosition);
- byte[] b = new byte[dataBytes.length - seekPosition];
- reader.readFully(b);
+ byte[] b = new byte[dataBytes.length - seekPosition];
+ reader.readFully(b);
- byte[] expected = Arrays.copyOfRange(dataBytes, seekPosition, dataBytes.length);
+ byte[] expected = Arrays.copyOfRange(dataBytes, seekPosition, dataBytes.length);
- assertArrayEquals(expected, b);
+ assertArrayEquals(expected, b);
- assertTrue(reader.isEOF());
-
- reader.close();
+ assertTrue(reader.isEOF());
+ }
}
- @Test(expected = ChecksummedRandomAccessReader.CorruptFileException.class)
+ @Test(expected = CorruptFileException.class)
public void corruptionDetection() throws IOException
{
final File data = File.createTempFile("corruptionDetection", "data");
@@ -116,14 +116,14 @@
dataFile.write((byte) 5);
}
- RandomAccessReader reader = new ChecksummedRandomAccessReader.Builder(data, crc).build();
- byte[] b = new byte[expected.length];
- reader.readFully(b);
+ try (RandomAccessReader reader = ChecksummedRandomAccessReader.open(data, crc))
+ {
+ byte[] b = new byte[expected.length];
+ reader.readFully(b);
- assertArrayEquals(expected, b);
+ assertArrayEquals(expected, b);
- assertTrue(reader.isEOF());
-
- reader.close();
+ assertTrue(reader.isEOF());
+ }
}
}
diff --git a/test/unit/org/apache/cassandra/io/util/DiskOptimizationStrategyTest.java b/test/unit/org/apache/cassandra/io/util/DiskOptimizationStrategyTest.java
new file mode 100644
index 0000000..986df96
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/DiskOptimizationStrategyTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.io.util;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class DiskOptimizationStrategyTest
+{
+ @Test
+ public void testRoundingBufferSize()
+ {
+ DiskOptimizationStrategy strategy = new SsdDiskOptimizationStrategy(0.95);
+ assertEquals(4096, strategy.roundBufferSize(-1L));
+ assertEquals(4096, strategy.roundBufferSize(0));
+ assertEquals(4096, strategy.roundBufferSize(1));
+ assertEquals(4096, strategy.roundBufferSize(2013));
+ assertEquals(4096, strategy.roundBufferSize(4095));
+ assertEquals(4096, strategy.roundBufferSize(4096));
+ assertEquals(8192, strategy.roundBufferSize(4097));
+ assertEquals(8192, strategy.roundBufferSize(8191));
+ assertEquals(8192, strategy.roundBufferSize(8192));
+ assertEquals(12288, strategy.roundBufferSize(8193));
+ assertEquals(65536, strategy.roundBufferSize(65535));
+ assertEquals(65536, strategy.roundBufferSize(65536));
+ assertEquals(65536, strategy.roundBufferSize(65537));
+ assertEquals(65536, strategy.roundBufferSize(10000000000000000L));
+ }
+
+ @Test
+ public void testBufferSize_ssd()
+ {
+ DiskOptimizationStrategy strategy = new SsdDiskOptimizationStrategy(0.1);
+
+ assertEquals(4096, strategy.bufferSize(0));
+ assertEquals(4096, strategy.bufferSize(10));
+ assertEquals(4096, strategy.bufferSize(100));
+ assertEquals(4096, strategy.bufferSize(4096));
+ assertEquals(8192, strategy.bufferSize(4505)); // just < (4096 + 4096 * 0.1)
+ assertEquals(12288, strategy.bufferSize(4506)); // just > (4096 + 4096 * 0.1)
+
+ strategy = new SsdDiskOptimizationStrategy(0.5);
+ assertEquals(8192, strategy.bufferSize(4506)); // just > (4096 + 4096 * 0.1)
+ assertEquals(8192, strategy.bufferSize(6143)); // < (4096 + 4096 * 0.5)
+ assertEquals(12288, strategy.bufferSize(6144)); // = (4096 + 4096 * 0.5)
+ assertEquals(12288, strategy.bufferSize(6145)); // > (4096 + 4096 * 0.5)
+
+ strategy = new SsdDiskOptimizationStrategy(1.0); // never add a page
+ assertEquals(8192, strategy.bufferSize(8191));
+ assertEquals(8192, strategy.bufferSize(8192));
+
+ strategy = new SsdDiskOptimizationStrategy(0.0); // always add a page
+ assertEquals(8192, strategy.bufferSize(10));
+ assertEquals(8192, strategy.bufferSize(4096));
+ }
+
+ @Test
+ public void testBufferSize_spinning()
+ {
+ DiskOptimizationStrategy strategy = new SpinningDiskOptimizationStrategy();
+
+ assertEquals(4096, strategy.bufferSize(0));
+ assertEquals(8192, strategy.bufferSize(10));
+ assertEquals(8192, strategy.bufferSize(100));
+ assertEquals(8192, strategy.bufferSize(4096));
+ assertEquals(12288, strategy.bufferSize(4097));
+ }
+}
diff --git a/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
index ee33107..e70d706 100644
--- a/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
+++ b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
@@ -30,9 +30,6 @@
import org.junit.Test;
import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.utils.FBUtilities;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
diff --git a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
index 32ce554..e5cc1c3 100644
--- a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
@@ -52,12 +52,12 @@
private static final class Parameters
{
- public final long fileLength;
- public final int bufferSize;
+ final long fileLength;
+ final int bufferSize;
- public BufferType bufferType;
- public int maxSegmentSize;
- public boolean mmappedRegions;
+ BufferType bufferType;
+ int maxSegmentSize;
+ boolean mmappedRegions;
public byte[] expected;
Parameters(long fileLength, int bufferSize)
@@ -70,19 +70,19 @@
this.expected = "The quick brown fox jumps over the lazy dog".getBytes(FileUtils.CHARSET);
}
- public Parameters mmappedRegions(boolean mmappedRegions)
+ Parameters mmappedRegions(boolean mmappedRegions)
{
this.mmappedRegions = mmappedRegions;
return this;
}
- public Parameters bufferType(BufferType bufferType)
+ Parameters bufferType(BufferType bufferType)
{
this.bufferType = bufferType;
return this;
}
- public Parameters maxSegmentSize(int maxSegmentSize)
+ Parameters maxSegmentSize(int maxSegmentSize)
{
this.maxSegmentSize = maxSegmentSize;
return this;
@@ -132,23 +132,21 @@
final long SIZE = 1L << 32; // 2GB
Parameters params = new Parameters(SIZE, 1 << 20); // 1MB
- try(ChannelProxy channel = new ChannelProxy("abc", new FakeFileChannel(SIZE)))
+
+ try (ChannelProxy channel = new ChannelProxy("abc", new FakeFileChannel(SIZE));
+ FileHandle.Builder builder = new FileHandle.Builder(channel)
+ .bufferType(params.bufferType).bufferSize(params.bufferSize);
+ FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
{
- RandomAccessReader.Builder builder = RandomAccessReader.builder(channel)
- .bufferType(params.bufferType)
- .bufferSize(params.bufferSize);
+ assertEquals(channel.size(), reader.length());
+ assertEquals(channel.size(), reader.bytesRemaining());
+ assertEquals(Integer.MAX_VALUE, reader.available());
- try(RandomAccessReader reader = builder.build())
- {
- assertEquals(channel.size(), reader.length());
- assertEquals(channel.size(), reader.bytesRemaining());
- assertEquals(Integer.MAX_VALUE, reader.available());
+ assertEquals(channel.size(), reader.skip(channel.size()));
- assertEquals(channel.size(), reader.skip(channel.size()));
-
- assertTrue(reader.isEOF());
- assertEquals(0, reader.bytesRemaining());
- }
+ assertTrue(reader.isEOF());
+ assertEquals(0, reader.bytesRemaining());
}
}
@@ -283,19 +281,12 @@
private static void testReadFully(Parameters params) throws IOException
{
final File f = writeFile(params);
- try(ChannelProxy channel = new ChannelProxy(f))
+ try (FileHandle.Builder builder = new FileHandle.Builder(f.getPath())
+ .bufferType(params.bufferType).bufferSize(params.bufferSize))
{
- RandomAccessReader.Builder builder = RandomAccessReader.builder(channel)
- .bufferType(params.bufferType)
- .bufferSize(params.bufferSize);
- MmappedRegions regions = null;
- if (params.mmappedRegions)
- {
- regions = MmappedRegions.map(channel, f.length());
- builder.regions(regions);
- }
-
- try(RandomAccessReader reader = builder.build())
+ builder.mmapped(params.mmappedRegions);
+ try (FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
{
assertEquals(f.getAbsolutePath(), reader.getPath());
assertEquals(f.length(), reader.length());
@@ -314,9 +305,6 @@
assertTrue(reader.isEOF());
assertEquals(0, reader.bytesRemaining());
}
-
- if (regions != null)
- assertNull(regions.close(null));
}
}
@@ -334,8 +322,9 @@
assert f.exists();
- try(ChannelProxy channel = new ChannelProxy(f);
- RandomAccessReader reader = RandomAccessReader.builder(channel).build())
+ try (FileHandle.Builder builder = new FileHandle.Builder(f.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
{
assertEquals(f.getAbsolutePath(), reader.getPath());
assertEquals(expected.length(), reader.length());
@@ -364,8 +353,9 @@
assert f.exists();
- try(ChannelProxy channel = new ChannelProxy(f);
- RandomAccessReader reader = RandomAccessReader.builder(channel).build())
+ try (FileHandle.Builder builder = new FileHandle.Builder(f.getPath());
+ FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
{
assertEquals(expected.length() * numIterations, reader.length());
@@ -443,11 +433,12 @@
assert f.exists();
- try(final ChannelProxy channel = new ChannelProxy(f))
+ try (FileHandle.Builder builder = new FileHandle.Builder(f.getPath()))
{
final Runnable worker = () ->
{
- try(RandomAccessReader reader = RandomAccessReader.builder(channel).build())
+ try (FileHandle fh = builder.complete();
+ RandomAccessReader reader = fh.createReader())
{
assertEquals(expected.length, reader.length());
diff --git a/test/unit/org/apache/cassandra/io/util/SegmentedFileTest.java b/test/unit/org/apache/cassandra/io/util/SegmentedFileTest.java
deleted file mode 100644
index 03c10de..0000000
--- a/test/unit/org/apache/cassandra/io/util/SegmentedFileTest.java
+++ /dev/null
@@ -1,88 +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.io.util;
-
-import org.junit.Test;
-
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-
-import static org.junit.Assert.assertEquals;
-
-public class SegmentedFileTest
-{
- @Test
- public void testRoundingBufferSize()
- {
- assertEquals(4096, SegmentedFile.Builder.roundBufferSize(-1L));
- assertEquals(4096, SegmentedFile.Builder.roundBufferSize(0));
- assertEquals(4096, SegmentedFile.Builder.roundBufferSize(1));
- assertEquals(4096, SegmentedFile.Builder.roundBufferSize(2013));
- assertEquals(4096, SegmentedFile.Builder.roundBufferSize(4095));
- assertEquals(4096, SegmentedFile.Builder.roundBufferSize(4096));
- assertEquals(8192, SegmentedFile.Builder.roundBufferSize(4097));
- assertEquals(8192, SegmentedFile.Builder.roundBufferSize(8191));
- assertEquals(8192, SegmentedFile.Builder.roundBufferSize(8192));
- assertEquals(12288, SegmentedFile.Builder.roundBufferSize(8193));
- assertEquals(65536, SegmentedFile.Builder.roundBufferSize(65535));
- assertEquals(65536, SegmentedFile.Builder.roundBufferSize(65536));
- assertEquals(65536, SegmentedFile.Builder.roundBufferSize(65537));
- assertEquals(65536, SegmentedFile.Builder.roundBufferSize(10000000000000000L));
- }
-
- @Test
- public void testBufferSize_ssd()
- {
- DatabaseDescriptor.setDiskOptimizationStrategy(Config.DiskOptimizationStrategy.ssd);
- DatabaseDescriptor.setDiskOptimizationPageCrossChance(0.1);
-
- assertEquals(4096, SegmentedFile.Builder.bufferSize(0));
- assertEquals(4096, SegmentedFile.Builder.bufferSize(10));
- assertEquals(4096, SegmentedFile.Builder.bufferSize(100));
- assertEquals(4096, SegmentedFile.Builder.bufferSize(4096));
- assertEquals(8192, SegmentedFile.Builder.bufferSize(4505)); // just < (4096 + 4096 * 0.1)
- assertEquals(12288, SegmentedFile.Builder.bufferSize(4506)); // just > (4096 + 4096 * 0.1)
-
- DatabaseDescriptor.setDiskOptimizationPageCrossChance(0.5);
- assertEquals(8192, SegmentedFile.Builder.bufferSize(4506)); // just > (4096 + 4096 * 0.1)
- assertEquals(8192, SegmentedFile.Builder.bufferSize(6143)); // < (4096 + 4096 * 0.5)
- assertEquals(12288, SegmentedFile.Builder.bufferSize(6144)); // = (4096 + 4096 * 0.5)
- assertEquals(12288, SegmentedFile.Builder.bufferSize(6145)); // > (4096 + 4096 * 0.5)
-
- DatabaseDescriptor.setDiskOptimizationPageCrossChance(1.0); // never add a page
- assertEquals(8192, SegmentedFile.Builder.bufferSize(8191));
- assertEquals(8192, SegmentedFile.Builder.bufferSize(8192));
-
- DatabaseDescriptor.setDiskOptimizationPageCrossChance(0.0); // always add a page
- assertEquals(8192, SegmentedFile.Builder.bufferSize(10));
- assertEquals(8192, SegmentedFile.Builder.bufferSize(4096));
- }
-
- @Test
- public void testBufferSize_spinning()
- {
- DatabaseDescriptor.setDiskOptimizationStrategy(Config.DiskOptimizationStrategy.spinning);
-
- assertEquals(4096, SegmentedFile.Builder.bufferSize(0));
- assertEquals(8192, SegmentedFile.Builder.bufferSize(10));
- assertEquals(8192, SegmentedFile.Builder.bufferSize(100));
- assertEquals(8192, SegmentedFile.Builder.bufferSize(4096));
- assertEquals(12288, SegmentedFile.Builder.bufferSize(4097));
- }
-}
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/test/unit/org/apache/cassandra/net/Matcher.java
similarity index 69%
copy from src/java/org/apache/cassandra/io/util/ICompressedFile.java
copy to test/unit/org/apache/cassandra/net/Matcher.java
index e69487c..cd1b667 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/test/unit/org/apache/cassandra/net/Matcher.java
@@ -15,12 +15,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.cassandra.io.util;
+package org.apache.cassandra.net;
-import org.apache.cassandra.io.compress.CompressionMetadata;
+import java.net.InetAddress;
-public interface ICompressedFile
+/**
+ * Predicate based on intercepted, outgoing messange and the message's destination address.
+ */
+public interface Matcher<T>
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ /**
+ * @param obj intercepted outgoing message
+ * @param to destination address
+ */
+ public boolean matches(MessageOut<T> obj, InetAddress to);
}
diff --git a/test/unit/org/apache/cassandra/net/MatcherResponse.java b/test/unit/org/apache/cassandra/net/MatcherResponse.java
new file mode 100644
index 0000000..c8984eb
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/MatcherResponse.java
@@ -0,0 +1,208 @@
+/*
+ * 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.net;
+
+import java.net.InetAddress;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * Sends a response for an incoming message with a matching {@link Matcher}.
+ * The actual behavior by any instance of this class can be inspected by
+ * interacting with the returned {@link MockMessagingSpy}.
+ */
+public class MatcherResponse
+{
+ private final Matcher<?> matcher;
+ private final Set<Integer> sendResponses = new HashSet<>();
+ private final MockMessagingSpy spy = new MockMessagingSpy();
+ private final AtomicInteger limitCounter = new AtomicInteger(Integer.MAX_VALUE);
+ private IMessageSink sink;
+
+ MatcherResponse(Matcher<?> matcher)
+ {
+ this.matcher = matcher;
+ }
+
+ /**
+ * Do not create any responses for intercepted outbound messages.
+ */
+ public MockMessagingSpy dontReply()
+ {
+ return respond((MessageIn<?>)null);
+ }
+
+ /**
+ * Respond with provided message in reply to each intercepted outbound message.
+ * @param message the message to use as mock reply from the cluster
+ */
+ public MockMessagingSpy respond(MessageIn<?> message)
+ {
+ return respondN(message, Integer.MAX_VALUE);
+ }
+
+ /**
+ * Respond a limited number of times with the provided message in reply to each intercepted outbound message.
+ * @param response the message to use as mock reply from the cluster
+ * @param limit number of times to respond with message
+ */
+ public MockMessagingSpy respondN(final MessageIn<?> response, int limit)
+ {
+ return respondN((in, to) -> response, limit);
+ }
+
+ /**
+ * Respond with the message created by the provided function that will be called with each intercepted outbound message.
+ * @param fnResponse function to call for creating reply based on intercepted message and target address
+ */
+ public <T, S> MockMessagingSpy respond(BiFunction<MessageOut<T>, InetAddress, MessageIn<S>> fnResponse)
+ {
+ return respondN(fnResponse, Integer.MAX_VALUE);
+ }
+
+ /**
+ * Respond with message wrapping the payload object created by provided function called for each intercepted outbound message.
+ * The target address from the intercepted message will automatically be used as the created message's sender address.
+ * @param fnResponse function to call for creating payload object based on intercepted message and target address
+ * @param verb verb to use for reply message
+ */
+ public <T, S> MockMessagingSpy respondWithPayloadForEachReceiver(Function<MessageOut<T>, S> fnResponse, MessagingService.Verb verb)
+ {
+ return respondNWithPayloadForEachReceiver(fnResponse, verb, Integer.MAX_VALUE);
+ }
+
+ /**
+ * Respond a limited number of times with message wrapping the payload object created by provided function called for
+ * each intercepted outbound message. The target address from the intercepted message will automatically be used as the
+ * created message's sender address.
+ * @param fnResponse function to call for creating payload object based on intercepted message and target address
+ * @param verb verb to use for reply message
+ */
+ public <T, S> MockMessagingSpy respondNWithPayloadForEachReceiver(Function<MessageOut<T>, S> fnResponse, MessagingService.Verb verb, int limit)
+ {
+ return respondN((MessageOut<T> msg, InetAddress to) -> {
+ S payload = fnResponse.apply(msg);
+ if (payload == null)
+ return null;
+ else
+ return MessageIn.create(to, payload, Collections.emptyMap(), verb, MessagingService.current_version, MessageIn.createTimestamp());
+ },
+ limit);
+ }
+
+ /**
+ * Responds to each intercepted outbound message by creating a response message wrapping the next element consumed
+ * from the provided queue. No reply will be send when the queue has been exhausted.
+ * @param cannedResponses prepared payload messages to use for responses
+ * @param verb verb to use for reply message
+ */
+ public <T, S> MockMessagingSpy respondWithPayloadForEachReceiver(Queue<S> cannedResponses, MessagingService.Verb verb)
+ {
+ return respondWithPayloadForEachReceiver((MessageOut<T> msg) -> cannedResponses.poll(), verb);
+ }
+
+ /**
+ * Responds to each intercepted outbound message by creating a response message wrapping the next element consumed
+ * from the provided queue. This method will block until queue elements are available.
+ * @param cannedResponses prepared payload messages to use for responses
+ * @param verb verb to use for reply message
+ */
+ public <T, S> MockMessagingSpy respondWithPayloadForEachReceiver(BlockingQueue<S> cannedResponses, MessagingService.Verb verb)
+ {
+ return respondWithPayloadForEachReceiver((MessageOut<T> msg) -> {
+ try
+ {
+ return cannedResponses.take();
+ }
+ catch (InterruptedException e)
+ {
+ return null;
+ }
+ }, verb);
+ }
+
+ /**
+ * Respond a limited number of times with the message created by the provided function that will be called with
+ * each intercepted outbound message.
+ * @param fnResponse function to call for creating reply based on intercepted message and target address
+ */
+ public <T, S> MockMessagingSpy respondN(BiFunction<MessageOut<T>, InetAddress, MessageIn<S>> fnResponse, int limit)
+ {
+ limitCounter.set(limit);
+
+ assert sink == null: "destroy() must be called first to register new response";
+
+ sink = new IMessageSink()
+ {
+ public boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to)
+ {
+ // prevent outgoing message from being send in case matcher indicates a match
+ // and instead send the mocked response
+ if (matcher.matches(message, to))
+ {
+ spy.matchingMessage(message);
+
+ if (limitCounter.decrementAndGet() < 0)
+ return false;
+
+ synchronized (sendResponses)
+ {
+ // I'm not sure about retry semantics regarding message/ID relationships, but I assume
+ // sending a message multiple times using the same ID shouldn't happen..
+ assert !sendResponses.contains(id) : "ID re-use for outgoing message";
+ sendResponses.add(id);
+ }
+ MessageIn<?> response = fnResponse.apply(message, to);
+ if (response != null)
+ {
+ CallbackInfo cb = MessagingService.instance().getRegisteredCallback(id);
+ if (cb != null)
+ cb.callback.response(response);
+ else
+ MessagingService.instance().receive(response, id);
+ spy.matchingResponse(response);
+ }
+ return false;
+ }
+ return true;
+ }
+
+ public boolean allowIncomingMessage(MessageIn message, int id)
+ {
+ return true;
+ }
+ };
+ MessagingService.instance().addMessageSink(sink);
+
+ return spy;
+ }
+
+ /**
+ * Stops currently registered response from being send.
+ */
+ public void destroy()
+ {
+ MessagingService.instance().removeMessageSink(sink);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/net/MockMessagingService.java b/test/unit/org/apache/cassandra/net/MockMessagingService.java
new file mode 100644
index 0000000..0412759
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/MockMessagingService.java
@@ -0,0 +1,144 @@
+/*
+ * 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.net;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.function.Predicate;
+
+/**
+ * Starting point for mocking {@link MessagingService} interactions. Outgoing messages can be
+ * intercepted by first creating a {@link MatcherResponse} by calling {@link MockMessagingService#when(Matcher)}.
+ * Alternatively {@link Matcher}s can be created by using helper methods such as {@link #to(InetAddress)},
+ * {@link #verb(MessagingService.Verb)} or {@link #payload(Predicate)} and may also be
+ * nested using {@link MockMessagingService#all(Matcher[])} or {@link MockMessagingService#any(Matcher[])}.
+ * After each test, {@link MockMessagingService#cleanup()} must be called for free listeners registered
+ * in {@link MessagingService}.
+ */
+public class MockMessagingService
+{
+
+ private MockMessagingService()
+ {
+ }
+
+ /**
+ * Creates a MatcherResponse based on specified matcher.
+ */
+ public static MatcherResponse when(Matcher matcher)
+ {
+ return new MatcherResponse(matcher);
+ }
+
+ /**
+ * Unsubscribes any handlers added by calling {@link MessagingService#addMessageSink(IMessageSink)}.
+ * This should be called after each test.
+ */
+ public static void cleanup()
+ {
+ MessagingService.instance().clearMessageSinks();
+ }
+
+ /**
+ * Creates a matcher that will indicate if the target address of the outgoing message equals the
+ * provided address.
+ */
+ public static Matcher<InetAddress> to(String address)
+ {
+ try
+ {
+ return to(InetAddress.getByName(address));
+ }
+ catch (UnknownHostException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Creates a matcher that will indicate if the target address of the outgoing message equals the
+ * provided address.
+ */
+ public static Matcher<InetAddress> to(InetAddress address)
+ {
+ return (in, to) -> to == address || to.equals(address);
+ }
+
+ /**
+ * Creates a matcher that will indicate if the verb of the outgoing message equals the
+ * provided value.
+ */
+ public static Matcher<MessagingService.Verb> verb(MessagingService.Verb verb)
+ {
+ return (in, to) -> in.verb == verb;
+ }
+
+ /**
+ * Creates a matcher based on the result of the provided predicate called with the outgoing message.
+ */
+ public static <T> Matcher<T> message(Predicate<MessageOut<T>> fn)
+ {
+ return (msg, to) -> fn.test(msg);
+ }
+
+ /**
+ * Creates a matcher based on the result of the provided predicate called with the outgoing message's payload.
+ */
+ public static <T> Matcher<T> payload(Predicate<T> fn)
+ {
+ return (msg, to) -> fn.test(msg.payload);
+ }
+
+ /**
+ * Inverts boolean result of wrapped matcher.
+ */
+ public static <T> Matcher<T> not(Matcher<T> matcher)
+ {
+ return (o, to) -> !matcher.matches(o, to);
+ }
+
+ /**
+ * Indicates true in case all provided matchers returned true.
+ */
+ public static <T> Matcher<?> all(Matcher<?>... matchers)
+ {
+ return (MessageOut<T> out, InetAddress to) -> {
+ for (Matcher matcher : matchers)
+ {
+ if (!matcher.matches(out, to))
+ return false;
+ }
+ return true;
+ };
+ }
+
+ /**
+ * Indicates true in case at least a single provided matcher returned true.
+ */
+ public static <T> Matcher<?> any(Matcher<?>... matchers)
+ {
+ return (MessageOut<T> out, InetAddress to) -> {
+ for (Matcher matcher : matchers)
+ {
+ if (matcher.matches(out, to))
+ return true;
+ }
+ return false;
+ };
+ }
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java
new file mode 100644
index 0000000..ce94f33
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.net;
+
+import java.util.Collections;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.gms.EchoMessage;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.net.MockMessagingService.all;
+import static org.apache.cassandra.net.MockMessagingService.to;
+import static org.apache.cassandra.net.MockMessagingService.verb;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class MockMessagingServiceTest
+{
+ @BeforeClass
+ public static void initCluster() throws ConfigurationException
+ {
+ SchemaLoader.prepareServer();
+ StorageService.instance.initServer();
+ }
+
+ @Before
+ public void cleanup()
+ {
+ MockMessagingService.cleanup();
+ }
+
+ @Test
+ public void testRequestResponse() throws InterruptedException, ExecutionException
+ {
+ // echo message that we like to mock as incoming reply for outgoing echo message
+ MessageIn<EchoMessage> echoMessageIn = MessageIn.create(FBUtilities.getBroadcastAddress(),
+ EchoMessage.instance,
+ Collections.emptyMap(),
+ MessagingService.Verb.ECHO,
+ MessagingService.current_version,
+ MessageIn.createTimestamp()
+ );
+ MockMessagingSpy spy = MockMessagingService
+ .when(
+ all(
+ to(FBUtilities.getBroadcastAddress()),
+ verb(MessagingService.Verb.ECHO)
+ )
+ )
+ .respond(echoMessageIn);
+
+ MessageOut<EchoMessage> echoMessageOut = new MessageOut<>(MessagingService.Verb.ECHO, EchoMessage.instance, EchoMessage.serializer);
+ MessagingService.instance().sendRR(echoMessageOut, FBUtilities.getBroadcastAddress(), new IAsyncCallback()
+ {
+ public void response(MessageIn msg)
+ {
+ assertEquals(MessagingService.Verb.ECHO, msg.verb);
+ assertEquals(echoMessageIn.payload, msg.payload);
+ }
+
+ public boolean isLatencyForSnitch()
+ {
+ return false;
+ }
+ });
+
+ // we must have intercepted the outgoing message at this point
+ MessageOut<?> msg = spy.captureMessageOut().get();
+ assertEquals(1, spy.messagesIntercepted);
+ assertTrue(msg == echoMessageOut);
+
+ // and return a mocked response
+ assertEquals(1, spy.mockedMessageResponses);
+ }
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/net/MockMessagingSpy.java b/test/unit/org/apache/cassandra/net/MockMessagingSpy.java
new file mode 100644
index 0000000..80bdb39
--- /dev/null
+++ b/test/unit/org/apache/cassandra/net/MockMessagingSpy.java
@@ -0,0 +1,234 @@
+/*
+ * 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.net;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import junit.framework.AssertionFailedError;
+
+/**
+ * Allows inspecting the behavior of mocked messaging by observing {@link MatcherResponse}.
+ */
+public class MockMessagingSpy
+{
+ private static final Logger logger = LoggerFactory.getLogger(MockMessagingSpy.class);
+
+ public int messagesIntercepted = 0;
+ public int mockedMessageResponses = 0;
+
+ private final BlockingQueue<MessageOut<?>> interceptedMessages = new LinkedBlockingQueue<>();
+ private final BlockingQueue<MessageIn<?>> deliveredResponses = new LinkedBlockingQueue<>();
+
+ private static final Executor executor = Executors.newSingleThreadExecutor();
+
+ /**
+ * Returns a future with the first mocked incoming message that has been created and delivered.
+ */
+ public ListenableFuture<MessageIn<?>> captureMockedMessageIn()
+ {
+ return Futures.transform(captureMockedMessageInN(1), (List<MessageIn<?>> result) -> result.isEmpty() ? null : result.get(0));
+ }
+
+ /**
+ * Returns a future with the specified number mocked incoming messages that have been created and delivered.
+ */
+ public ListenableFuture<List<MessageIn<?>>> captureMockedMessageInN(int noOfMessages)
+ {
+ CapturedResultsFuture<MessageIn<?>> ret = new CapturedResultsFuture<>(noOfMessages, deliveredResponses);
+ executor.execute(ret);
+ return ret;
+ }
+
+ /**
+ * Returns a future that will indicate if a mocked incoming message has been created and delivered.
+ */
+ public ListenableFuture<Boolean> expectMockedMessageIn()
+ {
+ return expectMockedMessageIn(1);
+ }
+
+ /**
+ * Returns a future that will indicate if the specified number of mocked incoming message have been created and delivered.
+ */
+ public ListenableFuture<Boolean> expectMockedMessageIn(int noOfMessages)
+ {
+ ResultsCompletionFuture<MessageIn<?>> ret = new ResultsCompletionFuture<>(noOfMessages, deliveredResponses);
+ executor.execute(ret);
+ return ret;
+ }
+
+ /**
+ * Returns a future with the first intercepted outbound message that would have been send.
+ */
+ public ListenableFuture<MessageOut<?>> captureMessageOut()
+ {
+ return Futures.transform(captureMessageOut(1), (List<MessageOut<?>> result) -> result.isEmpty() ? null : result.get(0));
+ }
+
+ /**
+ * Returns a future with the specified number of intercepted outbound messages that would have been send.
+ */
+ public ListenableFuture<List<MessageOut<?>>> captureMessageOut(int noOfMessages)
+ {
+ CapturedResultsFuture<MessageOut<?>> ret = new CapturedResultsFuture<>(noOfMessages, interceptedMessages);
+ executor.execute(ret);
+ return ret;
+ }
+
+ /**
+ * Returns a future that will indicate if an intercepted outbound messages would have been send.
+ */
+ public ListenableFuture<Boolean> interceptMessageOut()
+ {
+ return interceptMessageOut(1);
+ }
+
+ /**
+ * Returns a future that will indicate if the specified number of intercepted outbound messages would have been send.
+ */
+ public ListenableFuture<Boolean> interceptMessageOut(int noOfMessages)
+ {
+ ResultsCompletionFuture<MessageOut<?>> ret = new ResultsCompletionFuture<>(noOfMessages, interceptedMessages);
+ executor.execute(ret);
+ return ret;
+ }
+
+ /**
+ * Returns a future that will indicate the absence of any intercepted outbound messages with the specifed period.
+ */
+ public ListenableFuture<Boolean> interceptNoMsg(long time, TimeUnit unit)
+ {
+ ResultAbsenceFuture<MessageOut<?>> ret = new ResultAbsenceFuture<>(interceptedMessages, time, unit);
+ executor.execute(ret);
+ return ret;
+ }
+
+ void matchingMessage(MessageOut<?> message)
+ {
+ messagesIntercepted++;
+ logger.trace("Received matching message: {}", message);
+ interceptedMessages.add(message);
+ }
+
+ void matchingResponse(MessageIn<?> response)
+ {
+ mockedMessageResponses++;
+ logger.trace("Responding to intercepted message: {}", response);
+ deliveredResponses.add(response);
+ }
+
+
+ private static class CapturedResultsFuture<T> extends AbstractFuture<List<T>> implements Runnable
+ {
+ private final int waitForResults;
+ private final List<T> results;
+ private final BlockingQueue<T> queue;
+
+ CapturedResultsFuture(int waitForResponses, BlockingQueue<T> queue)
+ {
+ this.waitForResults = waitForResponses;
+ results = new ArrayList<T>(waitForResponses);
+ this.queue = queue;
+ }
+
+ public void run()
+ {
+ try
+ {
+ while (results.size() < waitForResults)
+ results.add(queue.take());
+
+ set(results);
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError();
+ }
+ }
+ }
+
+ private static class ResultsCompletionFuture<T> extends AbstractFuture<Boolean> implements Runnable
+ {
+ private final int waitForResults;
+ private final BlockingQueue<T> queue;
+
+ ResultsCompletionFuture(int waitForResponses, BlockingQueue<T> queue)
+ {
+ this.waitForResults = waitForResponses;
+ this.queue = queue;
+ }
+
+ public void run()
+ {
+ try
+ {
+ for (int i = 0; i < waitForResults; i++)
+ {
+ queue.take();
+ }
+ set(true);
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError();
+ }
+ }
+ }
+
+ private static class ResultAbsenceFuture<T> extends AbstractFuture<Boolean> implements Runnable
+ {
+ private final BlockingQueue<T> queue;
+ private final long time;
+ private final TimeUnit unit;
+
+ ResultAbsenceFuture(BlockingQueue<T> queue, long time, TimeUnit unit)
+ {
+ this.queue = queue;
+ this.time = time;
+ this.unit = unit;
+ }
+
+ public void run()
+ {
+ try
+ {
+ T result = queue.poll(time, unit);
+ if (result != null)
+ setException(new AssertionFailedError("Received unexpected message: " + result));
+ else
+ set(true);
+ }
+ catch (InterruptedException e)
+ {
+ throw new AssertionError();
+ }
+ }
+ }
+}
diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
index 2de671c..f31ecaa 100644
--- a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
+++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
@@ -37,6 +37,7 @@
import org.apache.cassandra.db.marshal.*;
import org.apache.cassandra.index.TargetParser;
import org.apache.cassandra.thrift.ThriftConversion;
+import org.apache.cassandra.utils.*;
import static java.lang.String.format;
import static junit.framework.Assert.assertEquals;
@@ -290,7 +291,7 @@
for (String name : collectionColumnNames)
{
ColumnDefinition column = table.getColumnDefinition(bytes(name));
- table.recordColumnDrop(column);
+ table.recordColumnDrop(column, FBUtilities.timestampMicros());
table.removeColumnDefinition(column);
}
diff --git a/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java b/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
index 4c0751f..e051357 100644
--- a/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
+++ b/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
@@ -19,7 +19,6 @@
package org.apache.cassandra.serializers;
import org.apache.cassandra.db.marshal.SimpleDateType;
-import org.apache.cassandra.utils.Pair;
import org.junit.Test;
import java.nio.ByteBuffer;
diff --git a/test/unit/org/apache/cassandra/service/PaxosStateTest.java b/test/unit/org/apache/cassandra/service/PaxosStateTest.java
index 9ee91dd..8054c61 100644
--- a/test/unit/org/apache/cassandra/service/PaxosStateTest.java
+++ b/test/unit/org/apache/cassandra/service/PaxosStateTest.java
@@ -27,7 +27,6 @@
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.Util;
import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.db.rows.Row;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.gms.Gossiper;
diff --git a/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java b/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java
new file mode 100644
index 0000000..7eae7bb
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/ProtocolBetaVersionTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.transport.*;
+import org.apache.cassandra.transport.messages.*;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.fail;
+
+public class ProtocolBetaVersionTest extends CQLTester
+{
+ @BeforeClass
+ public static void setUp()
+ {
+ requireNetwork();
+ DatabaseDescriptor.setBatchSizeWarnThresholdInKB(1);
+ }
+
+ @Test
+ public void testProtocolBetaVersion() throws Exception
+ {
+ createTable("CREATE TABLE %s (pk int PRIMARY KEY, v int)");
+
+ try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.BETA_VERSION, true, new EncryptionOptions.ClientEncryptionOptions()))
+ {
+ client.connect(false);
+ for (int i = 0; i < 10; i++)
+ {
+ QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, v) VALUES (%s, %s)",
+ KEYSPACE,
+ currentTable(),
+ i, i), QueryOptions.DEFAULT);
+ client.execute(query);
+ }
+
+ QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s",
+ KEYSPACE,
+ currentTable()), QueryOptions.DEFAULT);
+ ResultMessage.Rows resp = (ResultMessage.Rows) client.execute(query);
+ assertEquals(10, resp.result.size());
+ }
+ catch (Exception e)
+ {
+ fail("No exceptions should've been thrown: " + e.getMessage());
+ }
+ }
+
+ @Test
+ public void unforcedProtocolVersionTest() throws Exception
+ {
+ try
+ {
+ SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.BETA_VERSION, false, new EncryptionOptions.ClientEncryptionOptions());
+ client.connect(false);
+ fail("Exception should have been thrown");
+ }
+ catch (Exception e)
+ {
+ assertEquals("Beta version of server used (5), but USE_BETA flag is not set",
+ e.getMessage());
+ }
+ }
+}
+
diff --git a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
index 865a173..260c507 100644
--- a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
+++ b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
@@ -29,7 +29,6 @@
import org.junit.BeforeClass;
import org.junit.Test;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.cql3.BatchQueryOptions;
import org.apache.cassandra.cql3.CQLStatement;
import org.apache.cassandra.cql3.QueryHandler;
@@ -38,7 +37,6 @@
import org.apache.cassandra.cql3.statements.BatchStatement;
import org.apache.cassandra.cql3.statements.ParsedStatement;
import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.dht.ByteOrderedPartitioner;
import org.apache.cassandra.exceptions.RequestExecutionException;
import org.apache.cassandra.exceptions.RequestValidationException;
import org.apache.cassandra.service.ClientState;
diff --git a/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java b/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java
index 599087c..2be7b08 100644
--- a/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java
+++ b/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java
@@ -34,9 +34,9 @@
@Test
public void testInvalidProtocolVersion() throws Exception
{
- // test using a protocol version higher than the current version
- testInvalidProtocolVersion(Server.CURRENT_VERSION + 1);
- // test using a protocol version lower than the lowest version
+ // test using a protocol 2 version higher than the current version (1 version higher is current beta)
+ testInvalidProtocolVersion(Server.CURRENT_VERSION + 2); //
+ // test using a protocol version lower than the lowest version
testInvalidProtocolVersion(Server.MIN_SUPPORTED_VERSION - 1);
}
diff --git a/test/unit/org/apache/cassandra/utils/BTreeTest.java b/test/unit/org/apache/cassandra/utils/BTreeTest.java
index a01ad2e..ec4cdb8 100644
--- a/test/unit/org/apache/cassandra/utils/BTreeTest.java
+++ b/test/unit/org/apache/cassandra/utils/BTreeTest.java
@@ -21,6 +21,7 @@
import java.util.concurrent.ThreadLocalRandom;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
import org.junit.Test;
import junit.framework.Assert;
@@ -128,6 +129,30 @@
checkResult(i, BTree.update(BTree.build(seq(i), noOp), CMP, seq(i), updateF));
}
+ @Test
+ public void testApplyForwards()
+ {
+ List<Integer> input = seq(71);
+ Object[] btree = BTree.build(input, noOp);
+
+ final List<Integer> result = new ArrayList<>();
+ BTree.<Integer>apply(btree, i -> result.add(i), false);
+
+ org.junit.Assert.assertArrayEquals(input.toArray(),result.toArray());
+ }
+
+ @Test
+ public void testApplyReverse()
+ {
+ List<Integer> input = seq(71);
+ Object[] btree = BTree.build(input, noOp);
+
+ final List<Integer> result = new ArrayList<>();
+ BTree.<Integer>apply(btree, i -> result.add(i), true);
+
+ org.junit.Assert.assertArrayEquals(Lists.reverse(input).toArray(),result.toArray());
+ }
+
/**
* Tests that the apply method of the <code>UpdateFunction</code> is only called once with each key update.
* (see CASSANDRA-8018).
diff --git a/test/unit/org/apache/cassandra/utils/BloomFilterTest.java b/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
index 2e76e0e..818af9c 100644
--- a/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
+++ b/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
@@ -20,16 +20,13 @@
import java.io.*;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
-import java.util.List;
import java.util.Random;
import java.util.Set;
import org.junit.*;
-import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Murmur3Partitioner;
import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
diff --git a/test/unit/org/apache/cassandra/utils/FreeRunningClock.java b/test/unit/org/apache/cassandra/utils/FreeRunningClock.java
new file mode 100644
index 0000000..83c8db7
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/FreeRunningClock.java
@@ -0,0 +1,46 @@
+/*
+ * 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.concurrent.TimeUnit;
+
+/**
+ * A freely adjustable clock that can be used for unit testing. See {@link Clock#instance} how to
+ * enable this class.
+ */
+public class FreeRunningClock extends Clock
+{
+ private long nanoTime = 0;
+
+ @Override
+ public long nanoTime()
+ {
+ return nanoTime;
+ }
+
+ @Override
+ public long currentTimeMillis()
+ {
+ return TimeUnit.NANOSECONDS.toMillis(nanoTime());
+ }
+
+ public void advance(long time, TimeUnit unit)
+ {
+ nanoTime += unit.toNanos(time);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/utils/MergeIteratorComparisonTest.java b/test/unit/org/apache/cassandra/utils/MergeIteratorComparisonTest.java
index 5f2de73..6d9d2f6 100644
--- a/test/unit/org/apache/cassandra/utils/MergeIteratorComparisonTest.java
+++ b/test/unit/org/apache/cassandra/utils/MergeIteratorComparisonTest.java
@@ -36,7 +36,6 @@
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.TimeUUIDType;
import org.apache.cassandra.db.marshal.UUIDType;
-import org.apache.cassandra.utils.MergeIterator.Candidate;
import org.apache.cassandra.utils.MergeIterator.Reducer;
public class MergeIteratorComparisonTest
diff --git a/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java b/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java
index 5bbe267..6f99603 100644
--- a/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java
+++ b/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java
@@ -21,11 +21,6 @@
*/
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@@ -34,13 +29,6 @@
import org.junit.Test;
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.DataOutputPlus;
-
-import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class OverlapIteratorTest
diff --git a/tools/stress/src/org/apache/cassandra/stress/Operation.java b/tools/stress/src/org/apache/cassandra/stress/Operation.java
index 16f6f04..92a54f9 100644
--- a/tools/stress/src/org/apache/cassandra/stress/Operation.java
+++ b/tools/stress/src/org/apache/cassandra/stress/Operation.java
@@ -20,8 +20,6 @@
import java.io.IOException;
-import com.google.common.util.concurrent.RateLimiter;
-
import org.apache.cassandra.stress.settings.SettingsLog;
import org.apache.cassandra.stress.settings.StressSettings;
import org.apache.cassandra.stress.util.JavaDriverClient;
diff --git a/tools/stress/src/org/apache/cassandra/stress/Stress.java b/tools/stress/src/org/apache/cassandra/stress/Stress.java
index 874f515..daa7303 100644
--- a/tools/stress/src/org/apache/cassandra/stress/Stress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/Stress.java
@@ -80,7 +80,7 @@
}
catch (IllegalArgumentException e)
{
- System.out.printf("%s\n", e.getMessage());
+ System.out.printf("%s%n", e.getMessage());
printHelpMessage();
return 1;
}
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressGraph.java b/tools/stress/src/org/apache/cassandra/stress/StressGraph.java
index ebaa0ae..3b383fa 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressGraph.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressGraph.java
@@ -21,7 +21,6 @@
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
-import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
index f079d35..e5f7703 100644
--- a/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
@@ -20,11 +20,7 @@
*/
package org.apache.cassandra.stress.generate.values;
-import com.datastax.driver.core.LocalDate;
import org.apache.cassandra.db.marshal.SimpleDateType;
-import org.joda.time.DateTimeZone;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
public class LocalDates extends Generator<Integer>
{
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
index 702b6dc..a89fe12 100644
--- a/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
@@ -20,7 +20,6 @@
*/
package org.apache.cassandra.stress.generate.values;
-import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.ShortType;
public class SmallInts extends Generator<Short>
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
index 35bac86..b33d6de 100644
--- a/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
@@ -20,7 +20,6 @@
*/
package org.apache.cassandra.stress.generate.values;
-import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.TimeType;
public class Times extends Generator<Long>
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
index 0fe3f35..e5d46ec 100644
--- a/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
@@ -20,15 +20,7 @@
*/
package org.apache.cassandra.stress.generate.values;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
import org.apache.cassandra.db.marshal.ByteType;
-import org.apache.cassandra.db.marshal.DecimalType;
-import org.apache.cassandra.db.marshal.IntegerType;
-import org.apache.cassandra.db.marshal.ShortType;
-import org.apache.cassandra.stress.generate.FasterRandom;
public class TinyInts extends Generator<Byte>
{
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/PartitionOperation.java b/tools/stress/src/org/apache/cassandra/stress/operations/PartitionOperation.java
index 784b2ac..93290fc 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/PartitionOperation.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/PartitionOperation.java
@@ -21,8 +21,6 @@
import java.util.ArrayList;
import java.util.List;
-import com.google.common.util.concurrent.RateLimiter;
-
import org.apache.cassandra.stress.Operation;
import org.apache.cassandra.stress.WorkManager;
import org.apache.cassandra.stress.generate.Distribution;
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
index 166d689..2e03c69 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
@@ -30,7 +30,6 @@
import com.datastax.driver.core.LocalDate;
import com.datastax.driver.core.PreparedStatement;
import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.stress.Operation;
import org.apache.cassandra.stress.generate.Row;
import org.apache.cassandra.stress.operations.PartitionOperation;
import org.apache.cassandra.stress.settings.StressSettings;
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/TokenRangeQuery.java b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/TokenRangeQuery.java
index 198f1f5..7a0f02d 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/TokenRangeQuery.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/TokenRangeQuery.java
@@ -26,8 +26,6 @@
import javax.naming.OperationNotSupportedException;
-import com.google.common.util.concurrent.RateLimiter;
-
import com.datastax.driver.core.ColumnMetadata;
import com.datastax.driver.core.PagingState;
import com.datastax.driver.core.ResultSet;
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java
index 33f6f80..4547a37 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java
@@ -30,7 +30,6 @@
import com.datastax.driver.core.*;
import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.stress.Operation;
import org.apache.cassandra.stress.generate.*;
import org.apache.cassandra.stress.generate.Row;
import org.apache.cassandra.stress.operations.PartitionOperation;
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
index 89b7871..a081e55 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsNode.java
@@ -42,9 +42,8 @@
try
{
String node;
- List<String> tmpNodes = new ArrayList<String>();
- BufferedReader in = new BufferedReader(new InputStreamReader(new FileInputStream(options.file.value())));
- try
+ List<String> tmpNodes = new ArrayList<>();
+ try (BufferedReader in = new BufferedReader(new InputStreamReader(new FileInputStream(options.file.value()))))
{
while ((node = in.readLine()) != null)
{
@@ -53,10 +52,6 @@
}
nodes = Arrays.asList(tmpNodes.toArray(new String[tmpNodes.size()]));
}
- finally
- {
- in.close();
- }
}
catch(IOException ioe)
{
@@ -177,13 +172,6 @@
public static Runnable helpPrinter()
{
- return new Runnable()
- {
- @Override
- public void run()
- {
- printHelp();
- }
- };
+ return SettingsNode::printHelp;
}
}
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
index 5c437a9..debe22a 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
@@ -28,7 +28,6 @@
import com.datastax.driver.core.exceptions.AlreadyExistsException;
import org.apache.cassandra.stress.util.JavaDriverClient;
import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.thrift.ConsistencyLevel;
import org.apache.cassandra.utils.ByteBufferUtil;
public class SettingsSchema implements Serializable
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/Timing.java b/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
index fa95fdb..a304db7 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/Timing.java
@@ -23,7 +23,6 @@
import java.util.*;
import java.util.concurrent.Callable;
-import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/tools/stress/test/unit/org/apache/cassandra/stress/settings/OptionReplicationTest.java
similarity index 64%
copy from src/java/org/apache/cassandra/io/util/ICompressedFile.java
copy to tools/stress/test/unit/org/apache/cassandra/stress/settings/OptionReplicationTest.java
index e69487c..803ee18 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/tools/stress/test/unit/org/apache/cassandra/stress/settings/OptionReplicationTest.java
@@ -15,12 +15,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.cassandra.io.util;
-import org.apache.cassandra.io.compress.CompressionMetadata;
+package org.apache.cassandra.stress.settings;
-public interface ICompressedFile
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class OptionReplicationTest
{
- ChannelProxy channel();
- CompressionMetadata getMetadata();
+ @Test
+ public void defaultsToReplicationFactorOfOne() throws Exception
+ {
+ OptionReplication defaults = new OptionReplication();
+ assertEquals(ImmutableMap.of("replication_factor", "1"), defaults.getOptions());
+ }
}
diff --git a/tools/stress/test/unit/org/apache/cassandra/stress/settings/SettingsNodeTest.java b/tools/stress/test/unit/org/apache/cassandra/stress/settings/SettingsNodeTest.java
new file mode 100644
index 0000000..ce56d27
--- /dev/null
+++ b/tools/stress/test/unit/org/apache/cassandra/stress/settings/SettingsNodeTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.stress.settings;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class SettingsNodeTest
+{
+ @Test
+ public void testDefaults() throws Exception
+ {
+ SettingsNode settingsNode = new SettingsNode(new SettingsNode.Options());
+ assertEquals(null, settingsNode.datacenter);
+ }
+
+ @Test
+ public void testOveridingDataCenter() throws Exception
+ {
+ SettingsNode.Options options = new SettingsNode.Options();
+ options.accept("datacenter=dc1");
+ SettingsNode settingsNode = new SettingsNode(options);
+ assertEquals("dc1", settingsNode.datacenter);
+ }
+}