diff --git a/CHANGES.txt b/CHANGES.txt index 7e0c1a98a71a..462c5f41d00e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +Future version (tbd) + * Require only MODIFY permission on base when updating table with MV (STAR-564) + 4.0.1 * Cleanup dependency scopes (CASSANDRA-16704) * Make JmxHistogram#getRecentValues() and JmxTimer#getRecentValues() thread-safe (CASSANDRA-16707) diff --git a/README.asc b/README.asc index f1270a83df50..913c6ce55fef 100644 --- a/README.asc +++ b/README.asc @@ -22,8 +22,8 @@ and running, and demonstrate some simple reads and writes. For a more-complete g First, we'll unpack our archive: - $ tar -zxvf apache-cassandra-$VERSION.tar.gz - $ cd apache-cassandra-$VERSION + $ tar -zxvf dse-db-$VERSION.tar.gz + $ cd dse-db-$VERSION After that we start the server. Running the startup script with the -f argument will cause Cassandra to remain in the foreground and log to standard out; it can be stopped with ctrl-C. diff --git a/bin/cassandra.in.sh b/bin/cassandra.in.sh index 58b4dd2896b2..c3ded52b6921 100644 --- a/bin/cassandra.in.sh +++ b/bin/cassandra.in.sh @@ -30,10 +30,10 @@ CLASSPATH="$CASSANDRA_CONF" # compiled classes. NOTE: This isn't needed by the startup script, # it's just used here in constructing the classpath. if [ -d $CASSANDRA_HOME/build ] ; then - #cassandra_bin="$CASSANDRA_HOME/build/classes/main" - cassandra_bin=`ls -1 $CASSANDRA_HOME/build/apache-cassandra*.jar` + #dse_db_bin="$CASSANDRA_HOME/build/classes/main" + dse_db_bin=`ls -1 $CASSANDRA_HOME/build/dse-db*.jar` - CLASSPATH="$CLASSPATH:$cassandra_bin" + CLASSPATH="$CLASSPATH:$dse_db_bin" fi # the default location for commitlogs, sstables, and saved caches @@ -112,16 +112,16 @@ JAVA_VERSION=11 if [ "$JVM_VERSION" = "1.8.0" ] ; then JVM_PATCH_VERSION=${jvmver#*_} if [ "$JVM_VERSION" \< "1.8" ] || [ "$JVM_VERSION" \> "1.8.2" ] ; then - echo "Cassandra 4.0 requires either Java 8 (update 151 or newer) or Java 11 (or newer). Java $JVM_VERSION is not supported." + echo "DSE DB 4.0 requires either Java 8 (update 151 or newer) or Java 11 (or newer). Java $JVM_VERSION is not supported." exit 1; fi if [ "$JVM_PATCH_VERSION" -lt 151 ] ; then - echo "Cassandra 4.0 requires either Java 8 (update 151 or newer) or Java 11 (or newer). Java 8 update $JVM_PATCH_VERSION is not supported." + echo "DSE DB 4.0 requires either Java 8 (update 151 or newer) or Java 11 (or newer). Java 8 update $JVM_PATCH_VERSION is not supported." exit 1; fi JAVA_VERSION=8 elif [ "$JVM_VERSION" \< "11" ] ; then - echo "Cassandra 4.0 requires either Java 8 (update 151 or newer) or Java 11 (or newer)." + echo "DSE DB 4.0 requires either Java 8 (update 151 or newer) or Java 11 (or newer)." exit 1; fi diff --git a/bin/cqlsh.py b/bin/cqlsh.py index 37f839d9d443..6ff6e333f0a2 100755 --- a/bin/cqlsh.py +++ b/bin/cqlsh.py @@ -141,12 +141,10 @@ def find_zip(libprefix): 'Error: %s\n' % (sys.executable, sys.path, e)) from cassandra.auth import PlainTextAuthProvider -from cassandra.cluster import Cluster from cassandra.cqltypes import cql_typename from cassandra.marshal import int64_unpack from cassandra.metadata import (ColumnMetadata, KeyspaceMetadata, TableMetadata, protect_name, protect_names) -from cassandra.policies import WhiteListRoundRobinPolicy from cassandra.query import SimpleStatement, ordered_dict_factory, TraceUnavailable from cassandra.util import datetime_from_timestamp @@ -157,7 +155,7 @@ def find_zip(libprefix): sys.path.insert(0, cqlshlibdir) from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling, cqlshhandling -from cqlshlib.copyutil import ExportTask, ImportTask +from cqlshlib.copyutil import ExportTask, ImportTask, ImportConversion from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN, RED, WHITE, FormattedValue, colorme) from cqlshlib.formatting import (DEFAULT_DATE_FORMAT, DEFAULT_NANOTIME_FORMAT, @@ -166,11 +164,20 @@ def find_zip(libprefix): from cqlshlib.tracing import print_trace, print_trace_session from cqlshlib.util import get_file_encoding_bomsize, trim_if_present +from cqlshlib.geotypes import patch_geotypes_import_conversion # nopep8 +from cqlshlib.daterangetype import patch_daterange_import_conversion # nopep + +from cqlshlib.driver import cluster_factory + +patch_geotypes_import_conversion(ImportConversion) +patch_daterange_import_conversion(ImportConversion) + DEFAULT_HOST = '127.0.0.1' DEFAULT_PORT = 9042 DEFAULT_SSL = False DEFAULT_CONNECT_TIMEOUT_SECONDS = 5 DEFAULT_REQUEST_TIMEOUT_SECONDS = 10 +DEFAULT_NO_FILE_IO = False DEFAULT_FLOAT_PRECISION = 5 DEFAULT_DOUBLE_PRECISION = 5 @@ -203,6 +210,8 @@ def find_zip(libprefix): parser.add_option("-u", "--username", help="Authenticate as user.") parser.add_option("-p", "--password", help="Authenticate using password.") parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.') +parser.add_option('-b', '--secure-connect-bundle', + help="Connect using secure connect bundle. If this option is specified host, port settings are ignored.") parser.add_option("-f", "--file", help="Execute commands from FILE, then exit") parser.add_option('--debug', action='store_true', help='Show additional debugging information') @@ -223,8 +232,15 @@ def find_zip(libprefix): help='Specify the connection timeout in seconds (default: %default seconds).') parser.add_option("--request-timeout", default=DEFAULT_REQUEST_TIMEOUT_SECONDS, dest='request_timeout', help='Specify the default request timeout in seconds (default: %default seconds).') +parser.add_option("--consistency-level", dest='consistency_level', + help='Specify the initial consistency level.') +parser.add_option("--serial-consistency-level", dest='serial_consistency_level', + help='Specify the initial serial consistency level.') parser.add_option("-t", "--tty", action='store_true', dest='tty', help='Force tty mode (command prompt).') +parser.add_option("--no-file-io", action='store_true', dest='no_file_io', + help='Disable cqlsh commands that perform file I/O.') +parser.add_option('--disable-history', action='store_true', help='Disable saving of history', default=False) optvalues = optparse.Values() (options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues) @@ -416,13 +432,18 @@ class Shell(cmd.Cmd): last_hist = None shunted_query_out = None use_paging = True + no_file_io = DEFAULT_NO_FILE_IO default_page_size = 100 + consistency_level = None + serial_consistency_level = None 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=None, keyspace=None, + secure_connect_bundle=None, + consistency_level=None, serial_consistency_level=None, tracing_enabled=False, expand_enabled=False, display_nanotime_format=DEFAULT_NANOTIME_FORMAT, display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT, @@ -436,8 +457,11 @@ def __init__(self, hostname, port, color=False, request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS, protocol_version=None, connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS, - is_subshell=False): + no_file_io=DEFAULT_NO_FILE_IO, + is_subshell=False, + debug=False): cmd.Cmd.__init__(self, completekey=completekey) + self.debug = debug self.hostname = hostname self.port = port self.auth_provider = None @@ -451,19 +475,35 @@ def __init__(self, hostname, port, color=False, self.tracing_enabled = tracing_enabled self.page_size = self.default_page_size self.expand_enabled = expand_enabled + + if not consistency_level: + raise Exception('Argument consistency_level must not be None') + if not serial_consistency_level: + raise Exception('Argument serial_consistency_level must not be None') + self.consistency_level = consistency_level + self.serial_consistency_level = serial_consistency_level + + self.secure_connect_bundle = secure_connect_bundle + if use_conn: self.conn = use_conn else: kwargs = {} if protocol_version is not None: kwargs['protocol_version'] = protocol_version - self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver, - auth_provider=self.auth_provider, - ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None, - load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]), - control_connection_timeout=connect_timeout, - connect_timeout=connect_timeout, - **kwargs) + self.conn = cluster_factory( + self.hostname, + port=self.port, + cql_version=cqlver, + auth_provider=self.auth_provider, + ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None, + control_connection_timeout=connect_timeout, + connect_timeout=connect_timeout, + secure_connect_bundle=secure_connect_bundle, + application_name=description, + application_version=version, + **kwargs) + self.owns_connection = not use_conn if keyspace: @@ -487,7 +527,9 @@ def __init__(self, hostname, port, color=False, self.session.default_timeout = request_timeout self.session.row_factory = ordered_dict_factory - self.session.default_consistency_level = cassandra.ConsistencyLevel.ONE + self.session.default_consistency_level = self.consistency_level + self.session.default_serial_consistency_level = self.serial_consistency_level + self.get_connection_versions() self.set_expanded_cql_version(self.connection_versions['cql']) @@ -519,12 +561,10 @@ def __init__(self, hostname, port, color=False, self.show_line_nums = True self.stdin = stdin self.query_out = sys.stdout - self.consistency_level = cassandra.ConsistencyLevel.ONE - self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL - self.empty_lines = 0 self.statement_error = False self.single_statement = single_statement + self.no_file_io = no_file_io self.is_subshell = is_subshell @property @@ -1547,6 +1587,10 @@ def do_copy(self, parsed): on a line by itself to end the data input. """ + if self.no_file_io: + self.printerr('No file I/O permitted') + return + ks = self.cql_unprotect_name(parsed.get_binding('ksname', None)) if ks is None: ks = self.current_keyspace @@ -1631,6 +1675,11 @@ def do_source(self, parsed): See also the --file option to cqlsh. """ + + if self.no_file_io: + self.printerr('No file I/O permitted') + return + fname = parsed.get_binding('fname') fname = os.path.expanduser(self.cql_unprotect_value(fname)) try: @@ -1646,6 +1695,8 @@ def do_source(self, parsed): username=username, password=password, encoding=self.encoding, stdin=f, tty=False, use_conn=self.conn, cqlver=self.cql_version, keyspace=self.current_keyspace, + consistency_level=self.consistency_level, + serial_consistency_level=self.serial_consistency_level, tracing_enabled=self.tracing_enabled, display_nanotime_format=self.display_nanotime_format, display_timestamp_format=self.display_timestamp_format, @@ -1656,6 +1707,7 @@ def do_source(self, parsed): max_trace_wait=self.max_trace_wait, ssl=self.ssl, request_timeout=self.session.default_timeout, connect_timeout=self.conn.connect_timeout, + no_file_io=self.no_file_io, is_subshell=True) # duplicate coverage related settings in subshell if self.coverage: @@ -1691,6 +1743,11 @@ def do_capture(self, parsed): To inspect the current capture configuration, use CAPTURE with no arguments. """ + + if self.no_file_io: + self.printerr('No file I/O permitted') + return + fname = parsed.get_binding('fname') if fname is None: if self.shunted_query_out is not None: @@ -1829,8 +1886,9 @@ def do_login(self, parsed): LOGIN () - Login using the specified username. If password is specified, it will be used - otherwise, you will be prompted to enter. + Login using the specified username. + If password is specified it should be wrapped with single quotes. + If not specified you will be prompted to enter. """ username = parsed.get_binding('username') password = parsed.get_binding('password') @@ -1841,13 +1899,16 @@ def do_login(self, parsed): auth_provider = PlainTextAuthProvider(username=username, password=password) - conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version, - protocol_version=self.conn.protocol_version, - auth_provider=auth_provider, - ssl_options=self.conn.ssl_options, - load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]), - control_connection_timeout=self.conn.connect_timeout, - connect_timeout=self.conn.connect_timeout) + conn = cluster_factory( + self.hostname, + port=self.port, + cql_version=self.conn.cql_version, + protocol_version=self.conn.protocol_version, + auth_provider=auth_provider, + ssl_options=self.conn.ssl_options, + control_connection_timeout=self.conn.connect_timeout, + connect_timeout=self.conn.connect_timeout, + secure_connect_bundle=self.secure_connect_bundle) if self.current_keyspace: session = conn.connect(self.current_keyspace) @@ -1858,6 +1919,7 @@ def do_login(self, parsed): session.default_timeout = self.session.default_timeout session.row_factory = self.session.row_factory session.default_consistency_level = self.session.default_consistency_level + session.default_serial_consistency_level = self.session.default_serial_consistency_level session.max_trace_wait = self.session.max_trace_wait # Update after we've connected in case we fail to authenticate @@ -1888,6 +1950,11 @@ def do_clear(self, parsed): do_cls = do_clear def do_debug(self, parsed): + + if self.no_file_io: + self.printerr('No file I/O permitted') + return + import pdb pdb.set_trace() @@ -2118,6 +2185,7 @@ def read_options(cmdlineargs, environment): optvalues.username = option_with_default(configs.get, 'authentication', 'username') optvalues.password = option_with_default(rawconfigs.get, 'authentication', 'password') optvalues.keyspace = option_with_default(configs.get, 'authentication', 'keyspace') + optvalues.secure_connect_bundle = option_with_default(configs.get, 'connection', 'secure_connect_bundle') optvalues.browser = option_with_default(configs.get, 'ui', 'browser', None) optvalues.completekey = option_with_default(configs.get, 'ui', 'completekey', DEFAULT_COMPLETEKEY) @@ -2147,12 +2215,17 @@ def read_options(cmdlineargs, environment): optvalues.ssl = option_with_default(configs.getboolean, 'connection', 'ssl', DEFAULT_SSL) optvalues.encoding = option_with_default(configs.get, 'ui', 'encoding', UTF8) + optvalues.consistency_level = option_with_default(configs.get, 'cql', 'consistency_level', None) + optvalues.serial_consistency_level = option_with_default(configs.get, 'cql', 'serial_consistency_level', 'SERIAL') + optvalues.tty = option_with_default(configs.getboolean, 'ui', 'tty', sys.stdin.isatty()) optvalues.protocol_version = option_with_default(configs.getint, 'protocol', 'version', None) 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 + optvalues.no_file_io = option_with_default(configs.getboolean, 'ui', 'no_file_io', DEFAULT_NO_FILE_IO) + optvalues.disable_history = option_with_default(configs.getboolean, 'history', 'disabled', False) (options, arguments) = parser.parse_args(cmdlineargs, values=optvalues) # Make sure some user values read from the command line are in unicode @@ -2161,6 +2234,29 @@ def read_options(cmdlineargs, environment): options.password = maybe_ensure_text(options.password) options.keyspace = maybe_ensure_text(options.keyspace) + serial_levels = [cassandra.ConsistencyLevel.SERIAL, cassandra.ConsistencyLevel.LOCAL_SERIAL] + + # If unspecified, set the proper defaut CL + default_cl = 'LOCAL_QUORUM' if options.secure_connect_bundle else 'ONE' + if options.consistency_level is None: + options.consistency_level = default_cl + + try: + cl = cassandra.ConsistencyLevel.name_to_value[options.consistency_level.upper()] + if cl in serial_levels: + raise KeyError + options.consistency_level = cl + except KeyError: + parser.error('"{}" is not a valid consistency level'.format(options.consistency_level)) + + try: + cl = cassandra.ConsistencyLevel.name_to_value[options.serial_consistency_level.upper()] + if cl not in serial_levels: + raise KeyError + options.serial_consistency_level = cl + except KeyError: + parser.error('"{}" is not a valid serial consistency level'.format(options.serial_consistency_level)) + hostname = option_with_default(configs.get, 'connection', 'hostname', DEFAULT_HOST) port = option_with_default(configs.get, 'connection', 'port', DEFAULT_PORT) @@ -2237,8 +2333,8 @@ def init_history(): readline.set_completer_delims(delims) -def save_history(): - if readline is not None: +def save_history(history_disabled=False): + if readline is not None and not history_disabled: try: readline.write_history_file(HISTORY) except IOError: @@ -2264,8 +2360,11 @@ def main(options, hostname, port): if options.debug: sys.stderr.write("Using CQL driver: %s\n" % (cassandra,)) sys.stderr.write("Using connect timeout: %s seconds\n" % (options.connect_timeout,)) + sys.stderr.write("Using consistency level: %s\n" % (cassandra.ConsistencyLevel.value_to_name[options.consistency_level],)) sys.stderr.write("Using '%s' encoding\n" % (options.encoding,)) sys.stderr.write("Using ssl: %s\n" % (options.ssl,)) + if options.secure_connect_bundle: + sys.stderr.write("Using secure connect bundle: %s\n" % (options.secure_connect_bundle, )) # create timezone based on settings, environment or auto-detection timezone = None @@ -2309,9 +2408,13 @@ def main(options, hostname, port): tty=options.tty, completekey=options.completekey, browser=options.browser, + debug=options.debug, protocol_version=options.protocol_version, cqlver=options.cqlversion, keyspace=options.keyspace, + secure_connect_bundle=options.secure_connect_bundle, + consistency_level=options.consistency_level, + serial_consistency_level=options.serial_consistency_level, display_timestamp_format=options.time_format, display_nanotime_format=options.nanotime_format, display_date_format=options.date_format, @@ -2323,6 +2426,7 @@ def main(options, hostname, port): single_statement=options.execute, request_timeout=options.request_timeout, connect_timeout=options.connect_timeout, + no_file_io=options.no_file_io, encoding=options.encoding) except KeyboardInterrupt: sys.exit('Connection aborted.') @@ -2330,8 +2434,6 @@ def main(options, hostname, port): sys.exit('Connection error: %s' % (e,)) except VersionNotSupported as e: sys.exit('Unsupported CQL version: %s' % (e,)) - if options.debug: - shell.debug = True if options.coverage: shell.coverage = True import signal @@ -2343,7 +2445,7 @@ def handle_sighup(): signal.signal(signal.SIGHUP, handle_sighup) shell.cmdloop() - save_history() + save_history(options.disable_history) if shell.batch_mode and shell.statement_error: sys.exit(2) diff --git a/build.xml b/build.xml index 95b237ee16c3..34a1b9a4d2da 100644 --- a/build.xml +++ b/build.xml @@ -14,7 +14,7 @@ See the License for the specific language governing permissions and limitations under the License. --> - @@ -25,9 +25,9 @@ - - - + + + @@ -66,7 +66,7 @@ - + @@ -84,14 +84,14 @@ value="https://repo.maven.apache.org/maven2/org/apache/maven/maven-ant-tasks" /> - + - + - - + + @@ -284,7 +284,7 @@ @@ -472,14 +472,14 @@ description="Define dependencies and dependency versions"> + description="The Apache Cassandra Project develops a highly scalable second-generation distributed database. DataStax, Inc. provides additional improvements on top of Apache Cassandra"> @@ -523,7 +523,7 @@ - + @@ -563,7 +563,7 @@ - + @@ -600,6 +600,7 @@ + @@ -648,13 +649,19 @@ - + + + + + + + @@ -707,8 +714,8 @@ - @@ -723,21 +730,25 @@ - + + + + + - + @@ -772,7 +783,7 @@ - + @@ -812,6 +823,12 @@ + + + + + + @@ -1004,7 +1021,7 @@ --> + description="Assemble DSE DB JAR files"> @@ -1018,16 +1035,16 @@ - + - + + description="Assemble DSE DB JAR files"> @@ -1053,7 +1070,7 @@ - + @@ -1063,7 +1080,7 @@ - + @@ -1131,7 +1148,7 @@ + description="Create DSE DB release artifacts"> @@ -1218,10 +1235,12 @@ - + + + @@ -1354,6 +1373,7 @@ more aggressively rather than waiting. See CASSANDRA-14922 for more details. --> + @@ -1423,6 +1443,7 @@ + @@ -1446,6 +1467,7 @@ + @@ -1466,6 +1488,25 @@ + + + + + + + + + + + + + + + + + + @@ -1486,6 +1527,7 @@ + @@ -1557,7 +1599,15 @@ - + + + + + + + + + @@ -1806,6 +1856,13 @@ + + + + + + + @@ -2140,6 +2197,36 @@ + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh index dd088df81fd7..5360c33e4dff 100644 --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -99,7 +99,7 @@ if [ $JAVA_VERSION -ge 11 ] ; then if [ "$?" = "1" ] ; then # [X] to prevent ccm from replacing this line # only add -Xlog:gc if it's not mentioned in jvm-server.options file mkdir -p ${CASSANDRA_LOG_DIR} - JVM_OPTS="$JVM_OPTS -Xlog:gc=info,heap*=trace,age*=debug,safepoint=info,promotion*=trace:file=${CASSANDRA_LOG_DIR}/gc.log:time,uptime,pid,tid,level:filecount=10,filesize=10485760" + JVM_OPTS="$JVM_OPTS -Xlog:gc=info,heap*=debug,age*=debug,safepoint=info,promotion*=debug:file=${CASSANDRA_LOG_DIR}/gc.log:time,uptime,pid,tid,level:filecount=10,filesize=10485760" fi else # Java 8 diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index ec15f1f40b6d..c117bd9993b0 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -546,7 +546,7 @@ concurrent_materialized_view_writes: 32 # # offheap_objects # off heap objects -memtable_allocation_type: heap_buffers +memtable_allocation_type: offheap_objects # Limit memory usage for Merkle tree calculations during repairs. The default # is 1/16th of the available heap. The main tradeoff is that smaller trees @@ -633,7 +633,7 @@ index_summary_resize_interval_in_minutes: 60 # buffers. Enable this to avoid sudden dirty buffer flushing from # impacting read latencies. Almost always a good idea on SSDs; not # necessarily on platters. -trickle_fsync: false +trickle_fsync: true trickle_fsync_interval_in_kb: 10240 # TCP port, for commands and data @@ -1243,18 +1243,6 @@ transparent_data_encryption_options: # SAFETY THRESHOLDS # ##################### -# When executing a scan, within or across a partition, we need to keep the -# tombstones seen in memory so we can return them to the coordinator, which -# will use them to make sure other replicas also know about the deleted rows. -# With workloads that generate a lot of tombstones, this can cause performance -# problems and even exaust the server heap. -# (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets) -# Adjust the thresholds here if you understand the dangers and want to -# scan more tombstones anyway. These thresholds may also be adjusted at runtime -# using the StorageService mbean. -tombstone_warn_threshold: 1000 -tombstone_failure_threshold: 100000 - # Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a # mechanism called replica filtering protection to ensure that results from stale replicas do # not violate consistency. (See CASSANDRA-8272 and CASSANDRA-15907 for more details.) This @@ -1275,19 +1263,6 @@ replica_filtering_protection: cached_rows_warn_threshold: 2000 cached_rows_fail_threshold: 32000 -# Log WARN on any multiple-partition batch size exceeding this value. 5kb per batch by default. -# Caution should be taken on increasing the size of this threshold as it can lead to node instability. -batch_size_warn_threshold_in_kb: 5 - -# Fail any multiple-partition batch exceeding this value. 50kb (10x warn threshold) by default. -batch_size_fail_threshold_in_kb: 50 - -# Log WARN on any batches not of type LOGGED than span across more partitions than this limit -unlogged_batch_across_partitions_warn_threshold: 10 - -# Log a warning when compacting partitions larger than this value -compaction_large_partition_warning_threshold_mb: 100 - # GC Pauses greater than 200 ms will be logged at INFO level # This threshold can be adjusted to minimize logging if necessary # gc_log_threshold_in_ms: 200 @@ -1419,6 +1394,10 @@ report_unconfirmed_repaired_data_mismatches: false # table_count_warn_threshold: 150 # keyspace_count_warn_threshold: 40 +# This is the page size used internally by aggregation queries. It aims to limit the memory used by aggregation +# queries when there is a lot of data to aggregate. +# aggregation_subpage_size_in_kb: 2048 + ######################### # EXPERIMENTAL FEATURES # ######################### @@ -1434,3 +1413,138 @@ enable_sasi_indexes: false # Enables creation of transiently replicated keyspaces on this node. # Transient replication is experimental and is not recommended for production use. enable_transient_replication: false + + # Emulates DataStax Constellation database-as-a-service defaults. + # + # When enabled, some defaults are modified to match those used by DataStax Constellation (DataStax cloud data + # platform). This includes (but is not limited to) stricter guardrails defaults. + # + # This can be used as an convenience to develop and test applications meant to run on DataStax Constellation. + # + # Warning: when enabled, the updated defaults reflect those of DataStax Constellation _at the time_ of the currently + # used DSE release. This is a best-effort emulation of said defaults. Further, all nodes must use the same + # config value. + # emulate_dbaas_defaults: false + + # Guardrails settings. + # guardrails: + # When executing a scan, within or across a partition, we need to keep the + # tombstones seen in memory so we can return them to the coordinator, which + # will use them to make sure other replicas also know about the deleted rows. + # With workloads that generate a lot of tombstones, this can cause performance + # problems and even exhaust the server heap. + # (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets) + # Adjust the thresholds here if you understand the dangers and want to + # scan more tombstones anyway. These thresholds may also be adjusted at runtime + # using the StorageService mbean. + # + # Default tombstone_warn_threshold is 1000, may differ if emulate_dbaas_defaults is enabled + # Default tombstone_failure_threshold is 100000, may differ if emulate_dbaas_defaults is enabled + # tombstone_warn_threshold: 1000 + # tombstone_failure_threshold: 100000 + + # Log a warning when compacting partitions larger than this value. + # Default value is 100mb, may differ if emulate_dbaas_defaults is enabled + # partition_size_warn_threshold_in_mb: 100 + + # Log WARN on any multiple-partition batch size that exceeds this value. 64kb per batch by default. + # Use caution when increasing the size of this threshold as it can lead to node instability. + # Default value is 64kb, may differ if emulate_dbaas_defaults is enabled + # batch_size_warn_threshold_in_kb: 64 + + # Fail any multiple-partition batch that exceeds this value. The calculated default is 640kb (10x warn threshold). + # Default value is 640kb, may differ if emulate_dbaas_defaults is enabled + # batch_size_fail_threshold_in_kb: 640 + + # Log WARN on any batches not of type LOGGED than span across more partitions than this limit. + # Default value is 10, may differ if emulate_dbaas_defaults is enabled + # unlogged_batch_across_partitions_warn_threshold: 10 + + # Failure threshold to prevent writing large column value into Cassandra. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # column_value_size_failure_threshold_in_kb: -1 + + # Failure threshold to prevent creating more columns per table than threshold. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # columns_per_table_failure_threshold: -1 + + # Failure threshold to prevent creating more fields in user-defined-type than threshold. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # fields_per_udt_failure_threshold: -1 + + # Warning threshold to warn when encountering larger size of collection data than threshold. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # collection_size_warn_threshold_in_kb: -1 + + # Warning threshold to warn when encountering more elements in collection than threshold. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # items_per_collection_warn_threshold: -1 + + # Whether read-before-write operation is allowed, eg. setting list element by index, removing list element + # by index. Note: LWT is always allowed. + # Default true to allow read before write operation, may differ if emulate_dbaas_defaults is enabled + # read_before_write_list_operations_enabled: true + + # Failure threshold to prevent creating more secondary index per table than threshold (does not apply to CUSTOM INDEX StorageAttachedIndex) + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # secondary_index_per_table_failure_threshold: -1 + + # Failure threshold for number of StorageAttachedIndex per table (only applies to CUSTOM INDEX StorageAttachedIndex) + # Default is 10 (same when emulate_dbaas_defaults is enabled) + # sai_indexes_per_table_failure_threshold: 10 + # + # Failure threshold for total number of StorageAttachedIndex across all keyspaces (only applies to CUSTOM INDEX StorageAttachedIndex) + # Default is 10 (same when emulate_dbaas_defaults is enabled) + # sai_indexes_total_failure_threshold: 100 + + # Failure threshold to prevent creating more materialized views per table than threshold. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # materialized_view_per_table_failure_threshold: -1 + + # Warn threshold to warn creating more tables than threshold. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # tables_warn_threshold: -1 + + # Failure threshold to prevent creating more tables than threshold. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # tables_failure_threshold: -1 + + # Preventing creating tables with provided configurations. + # Default all properties are allowed, may differ if emulate_dbaas_defaults is enabled + # table_properties_disallowed: + + # Whether to allow user-provided timestamp in write request + # Default true to allow user-provided timestamp, may differ if emulate_dbaas_defaults is enabled + # user_timestamps_enabled: true + + # Preventing query with provided consistency levels + # Default all consistency levels are allowed. + # write_consistency_levels_disallowed: + + # Failure threshold to prevent providing larger paging by bytes than threshold, also served as a hard paging limit + # when paging by rows is used. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # page_size_failure_threshold_in_kb: -1 + + # Failure threshold to prevent IN query creating size of cartesian product exceeding threshold, eg. + # "a in (1,2,...10) and b in (1,2...10)" results in cartesian product of 100. + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # in_select_cartesian_product_failure_threshold: -1 + + # Failure threshold to prevent IN query containing more partition keys than threshold + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # partition_keys_in_select_failure_threshold: -1 + + # Warning threshold to warn when local disk usage exceeding threshold. Valid values: (1, 100] + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # disk_usage_percentage_warn_threshold: -1 + + # Failure threshold to reject write requests if replica disk usage exceeding threshold. Valid values: (1, 100] + # Default -1 to disable, may differ if emulate_dbaas_defaults is enabled + # disk_usage_percentage_failure_threshold: -1 + + # Allows configuring max disk size of data directories when calculating thresholds for disk_usage_percentage_warn_threshold + # and disk_usage_percentage_failure_threshold. Valid values: (1, max available disk size of all data directories] + # Default -1 to disable and use the physically available disk size of data directories during calculations. + # may differ if emulate_dbaas_defaults is enabled + # disk_usage_max_disk_size_in_gb: -1 diff --git a/conf/cqlshrc.sample.cloud b/conf/cqlshrc.sample.cloud new file mode 100644 index 000000000000..62528670c48b --- /dev/null +++ b/conf/cqlshrc.sample.cloud @@ -0,0 +1,17 @@ +; Copyright DataStax, Inc. +; +; Licensed 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. +; +; Sample ~/.cqlshrc file with cloud configuration. +[connection] +secure_connect_bundle = /path/to/creds.zip diff --git a/conf/jvm11-server.options b/conf/jvm11-server.options index 7e784678535c..7d58d1c5f820 100644 --- a/conf/jvm11-server.options +++ b/conf/jvm11-server.options @@ -11,34 +11,34 @@ ### CMS Settings --XX:+UseConcMarkSweepGC --XX:+CMSParallelRemarkEnabled --XX:SurvivorRatio=8 --XX:MaxTenuringThreshold=1 --XX:CMSInitiatingOccupancyFraction=75 --XX:+UseCMSInitiatingOccupancyOnly --XX:CMSWaitDuration=10000 --XX:+CMSParallelInitialMarkEnabled --XX:+CMSEdenChunksRecordAlways +#-XX:+UseConcMarkSweepGC +#-XX:+CMSParallelRemarkEnabled +#-XX:SurvivorRatio=8 +#-XX:MaxTenuringThreshold=1 +#-XX:CMSInitiatingOccupancyFraction=75 +#-XX:+UseCMSInitiatingOccupancyOnly +#-XX:CMSWaitDuration=10000 +#-XX:+CMSParallelInitialMarkEnabled +#-XX:+CMSEdenChunksRecordAlways ## some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 --XX:+CMSClassUnloadingEnabled +#-XX:+CMSClassUnloadingEnabled ### G1 Settings ## Use the Hotspot garbage-first collector. -#-XX:+UseG1GC -#-XX:+ParallelRefProcEnabled +-XX:+UseG1GC +-XX:+ParallelRefProcEnabled # ## Have the JVM do less remembered set work during STW, instead ## preferring concurrent GC. Reduces p99.9 latency. -#-XX:G1RSetUpdatingPauseTimePercent=5 +-XX:G1RSetUpdatingPauseTimePercent=5 # ## Main G1GC tunable: lowering the pause target will lower throughput and vise versa. ## 200ms is the JVM default and lowest viable setting ## 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml. -#-XX:MaxGCPauseMillis=500 +-XX:MaxGCPauseMillis=500 ## Optional G1 Settings # Save CPU time on large (>= 16GB) heaps by delaying region scanning @@ -80,7 +80,7 @@ # Java 11 (and newer) GC logging options: # See description of https://bugs.openjdk.java.net/browse/JDK-8046148 for details about the syntax # The following is the equivalent to -XX:+PrintGCDetails -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=10M -#-Xlog:gc=info,heap*=trace,age*=debug,safepoint=info,promotion*=trace:file=/var/log/cassandra/gc.log:time,uptime,pid,tid,level:filecount=10,filesize=10485760 +-Xlog:gc=info,heap*=debug,age*=debug,safepoint=info,promotion*=debug:file=/var/log/cassandra/gc.log:time,uptime,pid,tid,level:filecount=10,filesize=10485760 # Notes for Java 8 migration: # diff --git a/conf/jvm8-server.options b/conf/jvm8-server.options index 6214669eabdc..14d02617cd9e 100644 --- a/conf/jvm8-server.options +++ b/conf/jvm8-server.options @@ -18,33 +18,33 @@ ################# ### CMS Settings --XX:+UseParNewGC --XX:+UseConcMarkSweepGC --XX:+CMSParallelRemarkEnabled --XX:SurvivorRatio=8 --XX:MaxTenuringThreshold=1 --XX:CMSInitiatingOccupancyFraction=75 --XX:+UseCMSInitiatingOccupancyOnly --XX:CMSWaitDuration=10000 --XX:+CMSParallelInitialMarkEnabled --XX:+CMSEdenChunksRecordAlways +#-XX:+UseParNewGC +#-XX:+UseConcMarkSweepGC +#-XX:+CMSParallelRemarkEnabled +#-XX:SurvivorRatio=8 +#-XX:MaxTenuringThreshold=1 +#-XX:CMSInitiatingOccupancyFraction=75 +#-XX:+UseCMSInitiatingOccupancyOnly +#-XX:CMSWaitDuration=10000 +#-XX:+CMSParallelInitialMarkEnabled +#-XX:+CMSEdenChunksRecordAlways ## some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 --XX:+CMSClassUnloadingEnabled +#-XX:+CMSClassUnloadingEnabled ### G1 Settings ## Use the Hotspot garbage-first collector. -#-XX:+UseG1GC -#-XX:+ParallelRefProcEnabled +-XX:+UseG1GC +-XX:+ParallelRefProcEnabled # ## Have the JVM do less remembered set work during STW, instead ## preferring concurrent GC. Reduces p99.9 latency. -#-XX:G1RSetUpdatingPauseTimePercent=5 +-XX:G1RSetUpdatingPauseTimePercent=5 # ## Main G1GC tunable: lowering the pause target will lower throughput and vise versa. ## 200ms is the JVM default and lowest viable setting ## 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml. -#-XX:MaxGCPauseMillis=500 +-XX:MaxGCPauseMillis=500 ## Optional G1 Settings # Save CPU time on large (>= 16GB) heaps by delaying region scanning diff --git a/doc/native_protocol_v4.1.spec b/doc/native_protocol_v4.1.spec new file mode 100644 index 000000000000..a10fd2404d8f --- /dev/null +++ b/doc/native_protocol_v4.1.spec @@ -0,0 +1,1212 @@ +# +# 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. +# + + CQL BINARY PROTOCOL v4.1 + + +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 (4 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. + + This document describes version 4 of the protocol. For the changes made since + version 3, 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. + 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). If either or both + of the tracing and warning flags are set, the custom payload will follow + those indicated elements in the frame body. If neither are set, the custom + payload will be the first value in the frame body. + 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. + + 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 where is a [short] representing + the option id and depends on that option (and can be + of size 0). The supported id (and the corresponding ) + 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 where and + are [string]. + [string multimap] A [short] n, followed by n pair where is a + [string] and is a [string list]. + [bytes map] A [short] n, followed by n pair where is a + [string] and is a [bytes]. + + +4. Messages + + Dependant on the flags specified in the header, the layout of the message body must be: + [][][] + where: + - is a UUID tracing ID, present if this is a request message and the Tracing flag is set. + - is a string list of warnings (if this is a request message and the Warning flag is set. + - is bytes map for the serialised custom payload present if this is one of the message types + which support custom payloads (QUERY, PREPARE, EXECUTE and BATCH) and the Custom payload flag is set. + - as defined below through sections 4 and 5. + +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. + - "NO_COMPACT": whether or not connection has to be established in compatibility + mode. This mode will make all Thrift and Compact Tables to be exposed as if + they were CQL Tables. This is optional; if not specified, the option will + not be used. + - "THROW_ON_OVERLOAD": In case of server overloaded with too many requests, by default the server puts + back pressure on the client connection. Instead, the server can send an OverloadedException error message back to + the client if this option is set to true. + - "PAGE_UNIT": a list of supported page units. + + +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: + + where is a [long string] representing the query and + must be + [[name_1]...[name_n]][][][][] + where: + - is the [consistency] level for the operation. + - 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: + 0x00000001: Values. If set, a [short] followed by [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. + 0x00000002: 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). + 0x00000004: Page_size. If set, is an [int] + controlling the desired page size of the result (in CQL3 rows or bytes). + See the section on paging (Section 8) for more details. + 0x00000008: With_paging_state. If set, should be present. + 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). + 0x00000010: With serial consistency. If set, should be + present. 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. + 0x00000020: With default timestamp. If set, should be present. + 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. + 0x00000040: 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. + 0x40000000: When set, the is provided in bytes rather than in rows. + + + 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: + + where is the prepared query ID. It's the [short bytes] returned as a + response to a PREPARE message. As for , 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: + ...[][] + where: + - is a [byte] indicating the type of batch to use: + - If == 0, the batch will be "logged". This is equivalent to a + normal CQL3 batch statement. + - If == 1, the batch will be "unlogged". + - If == 2, the batch will be a "counter" batch (and non-counter + statements will be rejected). + - 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 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, should be + present. 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, should be present. + 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 must be + preceded by a [string] 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]. + - is a [short] indicating the number of following queries. + - ... are the queries to execute. A must be of the + form: + []...[] + where: + - is a [byte] indicating whether the following query is a prepared + one or not. value must be either 0 or 1. + - depends on the value of . If == 0, it should be + a [long string] query string (as in QUERY, the query string might contain + bind markers). Otherwise (that is, if == 1), it should be a + [short bytes] representing a prepared query ID. + - is a [short] indicating the number (possibly 0) of following values. + - is the optional name of the following . It must be present + if and only if the 0x40 flag is provided for the batch. + - is the [value] to use for bound variable i (of bound variable + if the 0x40 flag is used). + - is the [consistency] level for the operation. + - is only present if the 0x10 flag is set. In that case, + 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: + + where: + - is composed of: + [][?...] + where: + - is an [int]. The bits of 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 . If not + set, 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 + will be present. The 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 is only composed of + these , the and optionally the + (depending on the Has_more_pages flag) but + no other information (so no nor ). + This will only ever be the case if this was requested + during the query (see QUERY and RESULT messages). + - is an [int] representing the number of columns selected + by the query that produced this result. It defines the number of + elements in and the number of elements for each row in . + - is present if the Global_tables_spec is set in + . It is composed of two [string] representing the + (unique) keyspace name and table name the columns belong to. + - specifies the columns returned in the query. There are + such column specifications that are composed of: + ()? + The initial and are two [string] and are only present + if the Global_tables_spec flag is not set. The is a + [string] and 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 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 ... + where: + - is a [string] representing the keyspace name this + UDT is part of. + - is a [string] representing the UDT name. + - is a [short] representing the number of fields of + the UDT, and thus the number of pairs + following + - is a [string] representing the name of the + i_th field of the UDT. + - is an [option] representing the type of the + i_th field of the UDT. + 0x0031 Tuple: the value is ... where is a [short] + representing the number of values in the type, and + are [option] representing the type of the i_th component + of the tuple + + - is an [int] representing the number of rows present in this + result. Those rows are serialized in the part. + - is composed of ... where m is . + Each is composed of ... where n is + and where is a [bytes] representing the value + returned for the jth column of the ith row. In other words, + is composed of ( * ) [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: + + where: + - is [short bytes] representing the prepared query ID. + - is composed of: + [...][?...] + where: + - is an [int]. The bits of 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 . If not + set, is not present. + - is an [int] representing the number of bind markers + in the prepared statement. It defines the number of + elements. + - is an [int] representing the number of + 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). + - is a short that represents the index of the bind marker + that corresponds to the partition key column in position i. + For example, a 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. + - is present if the Global_tables_spec is set in + . 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. + - specifies the bind markers in the prepared statement. + There are such column specifications, each with the + following format: + ()? + The initial and are two [string] that are only + present if the Global_tables_spec flag is not set. The 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 + 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 field. + + - is defined exactly the same as 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 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: + + 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 where: + - is a [string] representing the type of changed involved. + It will be one of "CREATED", "UPDATED" or "DROPPED". + - 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). + - depends on the preceding : + - If is "KEYSPACE", then will be a single [string] + representing the keyspace changed. + - If is "TABLE" or "TYPE", then + 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 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 "E<-scale>". + In other words, " * 10 ^ (-1 * )". + +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 value that indicate the desired + page size in CQL3 rows or bytes. + + If a positive value is provided for , the result set of the + RESULT message returned for the query will contain at most the + first rows or bytes 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 value. In that case, the 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 value is + ignored. + + In the previous protocol versions the page size was always provided in rows. Since 4.1 + the page size can be provided in bytes as well. Whether the page size is specified in + rows or bytes is controlled by query flags (see section 4.1.4 for details). + + Note to client implementors: + - While 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 + results. While the current implementation always respects + the exact value of , we reserve the right to return + slightly smaller or bigger pages in the future for performance reasons. + - The is specific to a protocol version and drivers should not + send a 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 [...] + (see 4.2.1 for details). The supported error codes, as well as any additional + information the message may contain after the 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 + + where: + is the [consistency] level of the query that triggered + the exception. + is an [int] representing the number of nodes that + should be alive to respect + 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 < ) + 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 + + where: + is the [consistency] level of the query having triggered + the exception. + is an [int] representing the number of nodes having + acknowledged the request. + is an [int] representing the number of replicas whose + acknowledgement is required to achieve . + 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. + - "CAS": the timeout occured during the Compare And Set write/update. + - "VIEW": the timeout occured when a write involves + VIEW update and failure to acqiure local view(MV) + lock for key within timeout + - "CDC": the timeout occured when cdc_total_space_in_mb is + exceeded when doing a write to data tracked by cdc. + 0x1200 Read_timeout: Timeout exception during a read request. The rest + of the ERROR message body will be + + where: + is the [consistency] level of the query having triggered + the exception. + is an [int] representing the number of nodes having + answered the request. + is an [int] representing the number of replicas whose + response is required to achieve . Please note that + it is possible to have >= if + is false. Also in the (unlikely) + case where is achieved but the coordinator node + times out while waiting for read-repair acknowledgement. + 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 + + where: + is the [consistency] level of the query having triggered + the exception. + is an [int] representing the number of nodes having + answered the request. + is an [int] representing the number of replicas whose + acknowledgement is required to achieve . + is an [int] representing the number of nodes that + experience a failure while executing the request. + 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 + + where: + is the keyspace [string] of the failed function + is the name [string] of the failed function + [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 + + where: + is the [consistency] level of the query having triggered + the exception. + is an [int] representing the number of nodes having + answered the request. + is an [int] representing the number of replicas whose + acknowledgement is required to achieve . + is an [int] representing the number of nodes that + experience a failure while executing the request. + 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. + - "CAS": the failure occured during the Compare And Set write/update. + - "VIEW": the failure occured when a write involves + VIEW update and failure to acqiure local view(MV) + lock for key within timeout + - "CDC": the failure occured when cdc_total_space_in_mb is + exceeded when doing a write to data tracked by cdc. + + 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 where: + is a [string] representing either the keyspace that + already exists, or the keyspace in which the table that + already exists is. +
is a [string] representing the name of the table that + already exists. If the query was attempting to create a + keyspace,
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 + + * Query flags (Section 4.1.4) includes a new flag 0x40000000 which denotes that + the page size is specified in bytes rather than in rows. diff --git a/doc/source/configuration/cass_yaml_file.rst b/doc/source/configuration/cass_yaml_file.rst index e3babbcd7fdf..a553da71686f 100644 --- a/doc/source/configuration/cass_yaml_file.rst +++ b/doc/source/configuration/cass_yaml_file.rst @@ -1861,13 +1861,14 @@ Log WARN on any batches not of type LOGGED than span across more partitions than *Default Value:* 10 -``compaction_large_partition_warning_threshold_mb`` +``partition_size_warn_threshold_in_mb`` --------------------------------------------------- Log a warning when compacting partitions larger than this value *Default Value:* 100 +` ``gc_log_threshold_in_ms`` -------------------------- *This option is commented out by default.* diff --git a/doc/source/development/dependencies.rst b/doc/source/development/dependencies.rst index 6dd1cc46bc8f..ec7b56e53a35 100644 --- a/doc/source/development/dependencies.rst +++ b/doc/source/development/dependencies.rst @@ -47,7 +47,7 @@ Troubleshooting and conflict resolution Here are some useful commands that may help you out resolving conflicts. * ``ant realclean`` - gets rid of the build directory, including build artifacts. -* ``mvn dependency:tree -f build/apache-cassandra-*-SNAPSHOT.pom -Dverbose -Dincludes=org.slf4j`` - shows transitive dependency tree for artifacts, e.g. org.slf4j. In case the command above fails due to a missing parent pom file, try running ``ant mvn-install``. -* ``rm ~/.m2/repository/org/apache/cassandra/apache-cassandra/`` - removes cached local Cassandra maven artifacts +* ``mvn dependency:tree -f build/dse-db-*-SNAPSHOT.pom -Dverbose -Dincludes=org.slf4j`` - shows transitive dependency tree for artifacts, e.g. org.slf4j. In case the command above fails due to a missing parent pom file, try running ``ant mvn-install``. +* ``rm ~/.m2/repository/com/datastax/dse/dse-db/`` - removes cached local Cassandra maven artifacts diff --git a/doc/unified_compaction.md b/doc/unified_compaction.md new file mode 100644 index 000000000000..201de72562d7 --- /dev/null +++ b/doc/unified_compaction.md @@ -0,0 +1,172 @@ + + +## Unified compaction strategy (UCS) + +This is a new compaction strategy that unifies tiered and leveled compaction strategies, adds sharding, lends itself to be reconfigured at any time and forms the basis for future compaction improvements including automatic adaptation to the workload. + +The strategy is based on the observation that tiered and levelled compaction can be generalized as the same thing if one observes that both form exponentially-growing levels based on the size of sstables (or non-overlapping sstable runs) and trigger a compaction when more than a given number of sstables are present on one level. + +UCS groups sstables in levels based on the logarithm of the sstable size, with +the fanout factor **F** as the base of the logarithm, and with each level triggering a compaction as soon as it has +**T** sstables. The choice of the parameters **F** and **T**, and of a minimum sstable size, determines the behaviour +of the strategy. This allows users to choose a levelled strategy by setting **T=2**, or a tiered strategy by choosing **T=F**. Because the two options are mutually exclusive, meet at **F=2** and form a space of options for choosing different ratios of read amplification (RA) vs write amplification (WA) (where levelled compaction improves reads at the expense of writes and approaches a sorted array as **F** increases, and tiered compaction favors writes at the expense of reads and approaches an unsorted log as **F** increases), we combine the two parameters into one integer value, **W**, and set them to be: +* If **W < 0** then **F = 2 - W** and **T = 2**. This means leveled compactions, high WA but low RA. +* If **W > 0** then set **F = 2 + W** and **T = F**. This means tiered compactions, low WA but high RA. +* If **W = 0** then **F = T = 2**. This is the middle ground, leveled and tiered compactions behave identically. + +Further, because levels can choose different values of **W**, levels can behave differently. For example level +zero could behave like STCS but higher levels could behave more and more like LCS. + +This strategy also introduces compaction shards. Data is partitioned in independent shards that can be compacted in parallel. Shards are defined by splitting the token ranges for which the node is responsible into equally-sized sections. + +## Size based levels + +Let's explore more closely how sstables are grouped into levels. + +Given: + +- the fanout factor **F** +- the sstable flush size **m** (i.e. the average size of sstables written when a memtable is flushed) + +then the level **L** for an sstable of size **s** is calculated as follows: + +![L = log_F (s/m)](unified_compaction_level_formula.svg) + +This means that sstables are assigned to levels as follows: + +|Level|Min sstable size|Max sstable size| +|---|---|---| +|0|0|**m∙F**| +|1|**m∙F**|**m∙F2**| +|2|**m∙F2**|**m∙F3**| +|3|**m∙F3**|**m∙F4**| +|...|...|...| +|N|**m∙Fn**|**m∙Fn+1**| + +If we define **T** as the number of sstables in a level that triggers a compaction, then: + +* **T = 2** means the strategy is using a leveled merged policy. An sstable enters level **n** with size **>=mFn**. + When another sstable enters (also with size **>=mFn**) they compact and form a new table with size + **~2mFn**, which keeps the result in the same level for **F > 2**. After this repeats at least **F-2** + more times (i.e. F tables enter the level altogether), the compaction result grows to **>= mFn+1** + and enters the next level. +* **T = F** means the strategy is using a tiered merge policy. After **F** sstables enter level **n**, each of size **>=mFn**, they are compacted together, resulting in an sstable of size **>=mFn+1** which belongs to the next level. + +Note that the above ignores overwrites and deletions. Given knowledge of the expected proportion of overwrites/deletion, they can also be accounted for (this is not implemented at this time). + +For leveled strategies, the write amplification will be proportional to **F-1** times the number of levels whilst +for tiered strategies it will be proportional only to the number of levels. On the other hand, the read +amplification will be proportional to the number of levels for leveled strategies and to **F-1** times the number +of levels for tiered strategies. + +The number of levels for our size based scheme can be calculated by substituting the maximal dataset size **D** in our +equation above, giving a maximal number of levels inversely proportional to the logarithm of **F**. + +Therefore when we try to control the overheads of compaction on the database, we have a space of choices for the strategy +that range from: + +* leveled compaction (**T=2**) with high **F** - low number of levels, high read efficiency, high write cost, + moving closer to the behaviour of a sorted array as **F** increases; +* compaction with **T = F = 2** where leveled is the same as tiered and we have a middle ground with logarithmically + increasing read and write costs; +* tiered compaction (**T=F**) with high **F** - very high number of sstables, low read efficiency and low write cost, + moving closer to an unsorted log as **F** increases. + +## Sharding + +Sharding is used to reduce the size of the biggest files that are produced by compaction by splitting sstables at selected token boundaries when the data grows above a given size. This helps both with the maximum space overhead that compaction will require, as well as the number of concurrent compactions that can be executed. + +The number of required shards is specified in the compaction options. Based on that, the strategy will select shard boundaries which split the token space handled by the node in equal portions. The flush size **m** used in the calculations above is also divided by the number of shards. + +When flushing or when compacting, output sstables will be split along the boundaries of compaction shards as long as +they are at least as large as a minimum sstable size specified in the compaction options. If sstables are smaller than this size, then they will continue into the next shard. +The aim is to avoid sstables that are excessively small. For example, if there are four shards +and if the flush size is twice the minimum sstable size, then assuming uniform data distribution (no hot partitions), +flushing will create 2 sstables. The first sstable will be in the first shard and the second sstable will likely be +in the third shard. + +This means that some sstables effectively span several shards. We assign such sstables to the shard that contains their start position, but divide the size that we use for the level calculation by the number of spanned shards to reflect the fact that they contain less shard-specific data. This avoids problems with the result of compaction not advancing to the required level because it has shed data belonging to a different shard. + +For the example above and a fan factor of 2, the sstables in shards 1 and 3 belong to level 0, because their size is **m * 2** (where **m** is the shard-adjusted flush size), but taking into account that they each span 2 shards, we use the effective size of **m * 2 / 2**. Compacting them with another flushed pair will likely result in 4 sstables of the same size, one in each of the four shards. They, however, will all belong to level 1, as their effective size is **m * 2**. + +## Selecting compactions to run + +Because of sharding, UCS can do more compactions in parallel. However, it will often not use all available compaction threads. + +The reason for this is that UCS splits the available compaction threads equally among the levels of the compaction hierarchy. For example, if there are 16 compaction threads and we have no other work to do but compact the highest level 5, we can only use up to ⌈16/6⌉ = 3 threads to do compactions on level 5. If we permit more, we risk starving the lower levels of resources, which can result in sstables accumulating on level 0 for as long as that compaction takes, which can be a very long time, and breaking our read amplification expectations. + +In theory each level requires an equal amount of processing resources: for tiered compaction, every piece of data that enters the system goes through one compaction for each level, and for leveled - through **F-1**. Because of this UCS reserves an equal number of compaction threads for each level, and assign tasks to the remainder of threads randomly. + +Make sure the number of compaction threads is greater than the number of expected levels to ensure compaction runs smoothly. + +## Differences with STCS and LCS + +Note that there are some differences between the tiered flavors of UCS (UCS-tiered) and STCS, and between the leveled flavors of UCS (UCS-leveled) and LCS. + +#### UCS-tiered vs STCS + +SizeTieredCompactionStrategy is pretty close to UCS. However, it defines buckets/levels by looking for sstables of similar size. This can result in some odd selections of buckets, possibly spanning sstables of wildly different sizes, while UCS's selection is more stable and predictable. + +STCS triggers a compaction when it finds at least `min_threshold` sstables on some bucket, and it compacts between `min_threshold` and `max_threshold` sstables from that bucket at a time. `min_threshold` is equivalent to UCS's **T = F = W + 2**. UCS drops the upper limit as we have seen that compaction is still efficient with very large numbers of sstables. + +If there are multiple choices to pick SSTables within a bucket, STCS groups them by size while UCS groups them by timestamp. Because of that, STCS easily loses time order and makes whole table expiration less efficient. + +#### UCS-leveled vs LCS + +On first glance LeveledCompactionStrategy look very different in behaviour compared to UCS. + +LCS keeps multiple sstables per level which form a sorted run of non-overlapping sstables of small fixed size. So physical sstables on increasing levels increase in number (by a factor of `fanout_size`) instead of size. LCS does that to reduce space amplification and to ensure shorter compaction operations. When it finds that the combined size of a run on a level is higher than expected, it selects some sstables to compact with overlapping ones from the next level of the hierarchy. This eventually pushes the size of the next level over its size limit and triggers higher-level operations. + +UCS-leveled keeps one sstable per sharded level in the physical sense. So sstables on increasing levels increase in size (by a factor of **F**, see the **Size based levels** section above). UCS-leveled triggers a compaction when it finds a second sstable on some sharded level. It compacts the two sstables on that level, and the result most often ends up on that level too, but eventually it reaches sufficient size for the next level. This is the same time as a run in LCS would outgrow its size, thus compactions are in effect triggered at the same time as LCS would trigger them. + +The two approaches end up with a very similar effect, with the added benefit for UCS that sstables are structured in a way that can be easily switched to UCS-tiered or a different set values for the UCS parameters. + +UCS deals with the problem of space amplification by sharding on specific token boundaries. LCS's splitting of sstables on a fixed size means that the boundaries usually fall inside sstables on the next level, which tends to cause these sstables to be compacted more often than strictly necessary. This is not acceptable if we need tight write amplification control (i.e. this solution suits UCS-leveled, but not UCS-tiered and is thus not general enough for UCS). + +## Configuration + +UCS accepts these compaction strategy parameters: + +* **static_scaling_parameters**. Typically this will be a single integer, specifying **W** for all levels of the hierarchy. Positive values specify tiered compaction, and negative specify leveled, with fan factor **|W|+2**. Increasing **W** improves write amplification at the expense of reads, and decreasing it improves reads at the expense of writes. The default value is 2, which should be roughly equivalent to using STCS with the default threshold of 4. To use the equivalent of LCS with its default fan factor of 10, set this to -8.
+ The option also accepts passing a list of integers separated by a comma, in which case different values may be passed for the levels of the hierarchy. The first value will be used to set the value of + **W** for the first level, the second for the second level, and so on. The last value in this list will also be used for + all remaining levels. + +* **num_shards**. This is the number of shards. It is recommended that users set this value. More shards means more parallelism and smaller sstables at the higher levels at the expense of somewhat higher CPU usage. + By default, 10 would be used for single disk. + If JBOD / multi-drive, it would be 10 * disks. For example, if there are 5 disks, there would be 50 shards. + With data size 10 TB, the shard size would be 200 GB, which is an upper bound for the size of the largest sstables and compaction operations. + +* **min_sstable_size_in_mb**. This is the minimum sstable size in MB under which data will not be split on shard boundaries, by default 100. Higher values mean fewer sstables on disk and larger compaction operations on the lowest levels of the hierarchy. Storage-attached secondary indexes will work better with higher minimum sstable sizes. + +* **dataset_size_in_gb**. This is the target dataset size, by default the minimum total space for all the data file directories. + This is used to calculate the number of levels and therefore the theoretical read and write amplification. + It doesn't need to be very accurate but it is recommended + that it should be set to a value that is close to the target local dataset size, within a few GBs. + If not given, the database will use the total space on the devices containing the data directories, adjusting for the fact that data is equally split among them. + +* **max_space_overhead**. The maximum permitted space overhead as a fraction of the dataset size, by default 0.2 i.e. 20%. This cannot be smaller than 1/num_shards and limits the extra space that is required to complete compactions. UCS will only run compactions that will not overrun this limit. E.g. for datasize of 10TB and 20% max overhead, if a 1.1TB compaction is currently running, it will only start the 1.1TB one in the next shard after it completes. This also means that to prevent running out of space UCS will never start compactions that are larger than the limit by themselves; a warning will be issued if this happens as it may cause performance to deteriorate. + +* **expired_sstable_check_frequency_seconds**. Determines how often to check for expired SSTables, 10 minutes by default. + +* **unsafe_aggressive_sstable_expiration**. Expired sstables will be dropped without checking if their data is shadowing other sstables, by default false. This flag can only be enabled if `cassandra.allow_unsafe_aggressive_sstable_expiration` is true. Turning this flag can cause correctness issues, e.g. re-appearing of deleted data. See discussions in CASSANDRA-13418, DB-902 for valid use cases and potential problems. + +In **cassandra.yaml**: + +* **concurrent_compactors**. The number of compaction threads available. Set this to a large number, at minimum the number of expected levels of the compaction hierarchy to make sure that each level is given a dedicated compaction thread. This will avoid latency spikes caused by lower levels of the compaction hierarchy not getting a chance to run. diff --git a/doc/unified_compaction_level_formula.svg b/doc/unified_compaction_level_formula.svg new file mode 100644 index 000000000000..492fb81a825c --- /dev/null +++ b/doc/unified_compaction_level_formula.svg @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/eclipse_compiler.properties b/eclipse_compiler.properties index e1f28021c5f2..c0d02fb6397b 100644 --- a/eclipse_compiler.properties +++ b/eclipse_compiler.properties @@ -5,7 +5,10 @@ # # Autoclosables not in try-with-references org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=error -org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=error + +# the analysis is too shallow and forces to add supressions everywhere, which in turn may hide the real problems +# which could be detected by much smarter analysers (like the one in IntelliJ) +org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore org.eclipse.jdt.core.compiler.problem.unclosedCloseable=ignore #Ignore and disable all other checks too keep the logs clean diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py index 3bcd26a98362..a81d16721e20 100644 --- a/pylib/cqlshlib/copyutil.py +++ b/pylib/cqlshlib/copyutil.py @@ -42,7 +42,6 @@ from io import BytesIO, StringIO from select import select from uuid import UUID -from .util import profile_on, profile_off from six import ensure_str, ensure_text from six.moves import configparser @@ -50,23 +49,27 @@ from six.moves.queue import Queue from cassandra import OperationTimedOut -from cassandra.cluster import Cluster, DefaultConnection +from cassandra.cluster import DefaultConnection +from cassandra.connection import SniEndPoint from cassandra.cqltypes import ReversedType, UserType, BytesType, VarcharType from cassandra.metadata import protect_name, protect_names, protect_value from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy, DCAwareRoundRobinPolicy, FallthroughRetryPolicy from cassandra.query import BatchStatement, BatchType, SimpleStatement, tuple_factory from cassandra.util import Date, Time -from cqlshlib.util import profile_on, profile_off from cqlshlib.cql3handling import CqlRuleSet from cqlshlib.displaying import NO_COLOR_MAP +from cqlshlib.driver import cluster_factory from cqlshlib.formatting import format_value_default, CqlType, DateTimeFormat, EMPTY, get_formatter, BlobType from cqlshlib.sslhandling import ssl_settings +from cqlshlib.util import profile_on, profile_off + PROFILE_ON = False STRACE_ON = False DEBUG = False # This may be set to True when initializing the task IS_LINUX = platform.system() == 'Linux' +IS_WINDOWS = platform.system() == 'Windows' CopyOptions = namedtuple('CopyOptions', 'copy dialect unrecognized') @@ -494,6 +497,7 @@ def make_params(self): port=shell.port, ssl=shell.ssl, auth_provider=shell.auth_provider, + parent_cluster=shell.conn if not IS_WINDOWS else None, cql_version=shell.conn.cql_version, config_file=self.config_file, protocol_version=self.protocol_version, @@ -716,9 +720,9 @@ def make_range(prev, curr): def make_range_data(replicas=None): hosts = [] if replicas: - for r in replicas: - if r.is_up is not False and r.datacenter == local_dc: - hosts.append(r.address) + # when connected to a cloud cluster r.address is the proxy + # so we need to use host_id as correct in both cloud/not cloud + hosts = [r.host_id for r in replicas if r.is_up is not False and r.datacenter == local_dc] if not hosts: hosts.append(hostname) # fallback to default host if no replicas in current dc return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0, 'workerno': -1} @@ -1409,6 +1413,7 @@ def __init__(self, params, target): self.connect_timeout = params['connect_timeout'] self.cql_version = params['cql_version'] self.auth_provider = params['auth_provider'] + self.parent_cluster = params['parent_cluster'] self.ssl = params['ssl'] self.protocol_version = params['protocol_version'] self.config_file = params['config_file'] @@ -1676,23 +1681,36 @@ def connect(self, host): session.add_request() return session - new_cluster = Cluster( - contact_points=(host,), + endpoint = self._endpoint_for_host(host) + + new_cluster = cluster_factory( + endpoint, + whitelist_lbp=endpoint, + cloud=self.parent_cluster.cloud, port=self.port, cql_version=self.cql_version, protocol_version=self.protocol_version, auth_provider=self.auth_provider, - ssl_options=ssl_settings(host, self.config_file) if self.ssl else None, - load_balancing_policy=WhiteListRoundRobinPolicy([host]), + ssl_options=ssl_settings(endpoint, self.config_file) if self.ssl else None, default_retry_policy=ExpBackoffRetryPolicy(self), compression=None, control_connection_timeout=self.connect_timeout, connect_timeout=self.connect_timeout, idle_heartbeat_interval=0) + session = ExportSession(new_cluster, self) self.hosts_to_sessions[host] = session return session + def _endpoint_for_host(self, host): + endpoint = host + if isinstance(host, UUID): + for h in self.parent_cluster.metadata.all_hosts(): + if h.host_id == host: + endpoint = h.endpoint if isinstance(h.endpoint, SniEndPoint) else h.endpoint.address + break + return endpoint + def attach_callbacks(self, token_range, future, session): metadata = session.cluster.metadata ks_meta = metadata.keyspaces[self.ks] @@ -2353,8 +2371,9 @@ def __init__(self, params): @property def session(self): if not self._session: - cluster = Cluster( - contact_points=(self.hostname,), + cluster = cluster_factory( + self.hostname, + whitelist_lbp=False, port=self.port, cql_version=self.cql_version, protocol_version=self.protocol_version, @@ -2366,6 +2385,7 @@ def session(self): control_connection_timeout=self.connect_timeout, connect_timeout=self.connect_timeout, idle_heartbeat_interval=0, + cloud=self.parent_cluster.cloud, connection_class=ConnectionWrapper) self._session = cluster.connect(self.ks) diff --git a/pylib/cqlshlib/daterangetype.py b/pylib/cqlshlib/daterangetype.py new file mode 100644 index 000000000000..e052001683b6 --- /dev/null +++ b/pylib/cqlshlib/daterangetype.py @@ -0,0 +1,116 @@ +# 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. + +from datetime import datetime +from cassandra.util import DateRange, DateRangeBound, DateRangePrecision, OPEN_BOUND, ms_timestamp_from_datetime + + +def _raise_parse_error(msg): + """ + We need to throw ParseError which is defined in copyutil + since copyutil imports this file as part of it's initialization though, + importing it at the top of this file would create an import loop, so + we just import it locally when/if we need it + """ + from cqlshlib import copyutil + raise copyutil.ParseError(msg) + + +def _new_date_range_bound(datetime, precision): + millis = ms_timestamp_from_datetime(datetime) + return DateRangeBound(value=millis, precision=precision) + + +def _parse_date_range_bound(val): + if val == '*': + return OPEN_BOUND + else: + year = 1 + month = 1 + day = 1 + hour = 0 + minute = 0 + second = 0 + millisecond = 0 + precision = DateRangePrecision.MILLISECOND + + date_split = val.replace('Z', '').split('-') + if len(date_split) > 0: + year = int(date_split[0]) + precision = DateRangePrecision.YEAR + if len(date_split) > 1: + month = int(date_split[1]) + precision = DateRangePrecision.MONTH + if len(date_split) > 2: + day_split = date_split[2].split('T') + day = int(day_split[0]) + precision = DateRangePrecision.DAY + if len(day_split) > 1: + time_split = day_split[1].split(':') + if len(time_split) > 0: + hour = int(time_split[0]) + precision = DateRangePrecision.HOUR + if len(time_split) > 1: + minute = int(time_split[1]) + precision = DateRangePrecision.MINUTE + if len(time_split) > 2: + second_split = time_split[2].split('.') + second = int(second_split[0]) + precision = DateRangePrecision.SECOND + if len(second_split) > 1: + millisecond = int(second_split[1]) + precision = DateRangePrecision.MILLISECOND + + return _new_date_range_bound( + datetime=datetime(year=year, month=month, day=day, hour=hour, minute=minute, second=second, + microsecond=millisecond * 1000), + precision=precision) + + +def _convert_daterange(val): + if val.startswith('['): + if val.endswith(']'): + bounds = val[1:-1].split(' TO ') + if len(bounds) == 2: + lower_bound = _parse_date_range_bound(bounds[0].strip()).round_down() + upper_bound = _parse_date_range_bound(bounds[1].strip()).round_up() + return DateRange(lower_bound=lower_bound, upper_bound=upper_bound) + else: + _raise_parse_error("If date range starts with [ must contain ' TO '; got {}".format(val)) + else: + _raise_parse_error("If date range starts with [ must end with ]; got {}".format(val)) + else: + bound = _parse_date_range_bound(val).round_down() + return DateRange(value=bound) + + +def _patch_get_converters(klass): + original_method = klass._get_converter + + def new_method(self, cql_type): + if cql_type.typename == 'daterange': + return _convert_daterange + else: + return original_method(self, cql_type) + + klass._get_converter = new_method + + +def patch_daterange_import_conversion(klass): + """ + monkey patches cqlshlib.copyutil.ImportConversion to support DateRangeType + """ + _patch_get_converters(klass) diff --git a/pylib/cqlshlib/driver.py b/pylib/cqlshlib/driver.py new file mode 100644 index 000000000000..e0bc7cef951f --- /dev/null +++ b/pylib/cqlshlib/driver.py @@ -0,0 +1,153 @@ +# Copyright DataStax, Inc. +# +# Licensed 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 os +import random +import six +import stat +from cassandra.cluster import Cluster +from cassandra.connection import UnixSocketEndPoint +from cassandra.policies import WhiteListRoundRobinPolicy, RoundRobinPolicy +from cassandra.pool import HostDistance + + +class EndpointWhiteListRoundRobinPolicy(WhiteListRoundRobinPolicy): + """ + WhileListRoundRobinPolicy dedicated for the cloud connection. + + It is using endpoint instead of host address. + + Note: + we want to override _allowed_hosts and _allowed_hosts_resolved which is the reason + to not call direct super class init but rather the one from RoundRobinPolicy. + """ + + def __init__(self, hosts): + self._allowed_hosts = self._allowed_hosts_resolved = tuple(hosts) + RoundRobinPolicy.__init__(self) + + def populate(self, cluster, hosts): + self._live_hosts = frozenset(h for h in hosts if h.endpoint in self._allowed_hosts_resolved) + + if len(hosts) <= 1: + self._position = 0 + else: + self._position = random.randint(0, len(hosts) - 1) + + def distance(self, host): + if host.endpoint in self._allowed_hosts_resolved: + return HostDistance.LOCAL + else: + return HostDistance.IGNORED + + def on_up(self, host): + if host.endpoint in self._allowed_hosts_resolved: + RoundRobinPolicy.on_up(self, host) + + def on_add(self, host): + if host.endpoint in self._allowed_hosts_resolved: + RoundRobinPolicy.on_add(self, host) + + +def cluster_factory(host, whitelist_lbp=True, **kwargs): + """ + Cluster factory to create a cassandra or dse cluster for cqlsh. + + :param host: the host for the connection. This can be the hostname string or an EndPoint instance. + :param whitelist_lbp: Specify if a WhiteListRoundRobinPolicy should be applied. This can be set to True, + False or an EndPoint instance, which is used to configure the policy when + connecting to a cloud cluster. On a cloud cluster, the LBP has to be set after the + nodes discovery. Otherwise, the LBP is set as usual to the execution profile. + Default to True. + :kwargs: All other keyword arguments are passed to the Cluster constructor. + :return: A Cluster instance. + """ + + is_cloud_cluster = False + is_unix_socket_endpoint = False + endpoint = host + options = kwargs.copy() + + # Configure the cluster contact point type and address. + if is_unix_socket(host): + # update endpoint and load balancing policy for unix socket + endpoint = UnixSocketEndPoint(host) + is_unix_socket_endpoint = True + + # Determine if we are trying to connect to a cloud cluster + secure_connect_bundle = options.pop('secure_connect_bundle', None) + if secure_connect_bundle: + is_cloud_cluster = True + options['cloud'] = {'secure_connect_bundle': secure_connect_bundle} + elif 'cloud' in options: + is_cloud_cluster = options['cloud'] is not None + + # Build the Cluster instance + if is_cloud_cluster: + return _cloud_cluster_factory(whitelist_lbp, **options) + + if is_unix_socket_endpoint and 'port' in options: + del options['port'] + + contact_points = (endpoint,) + + if whitelist_lbp: + lbp_class = WhiteListRoundRobinPolicy + if is_unix_socket_endpoint: + lbp_class = EndpointWhiteListRoundRobinPolicy + + whitelist = [endpoint] if whitelist_lbp is True else [whitelist_lbp] + options['load_balancing_policy'] = lbp_class(whitelist) + + return Cluster(contact_points=contact_points, **options) + + +def is_unix_socket(hostname): + if isinstance(hostname, six.string_types) and os.path.exists(hostname): + mode = os.stat(hostname).st_mode + return stat.S_ISSOCK(mode) + return False + + +def _cloud_cluster_factory(whitelist_lbp, **kwargs): + """ + Create cloud cluster from given options. + Please notice that: + - cloud should be present in option + - contact_points, endpoint_factory, ssl_context, and ssl_options cannot be specified with a cloud configuration (will be removed) + - whitelist_lbp can be True (random contact point), False (no policy set), or specific endpoint + """ + options = kwargs.copy() + assert 'cloud' in options + + cloud_prohibited_options = ['contact_points', 'endpoint_factory', 'ssl_context', 'ssl_options'] + for opt_name in cloud_prohibited_options: + options.pop(opt_name, None) + + cluster = Cluster(**options) + + if whitelist_lbp: + + if whitelist_lbp is True: + # applying load balancing policy as we now know the contact points + contact_points = [random.choice(cluster.contact_points)] + else: + # An explicit host was specified + contact_points = [whitelist_lbp] + + for execution_profile in cluster.profile_manager.profiles.values(): + execution_profile.load_balancing_policy = EndpointWhiteListRoundRobinPolicy(contact_points) + cluster.endpoints_resolved = contact_points + + return cluster diff --git a/pylib/cqlshlib/geotypes.py b/pylib/cqlshlib/geotypes.py new file mode 100644 index 000000000000..bad2f2b85b1b --- /dev/null +++ b/pylib/cqlshlib/geotypes.py @@ -0,0 +1,121 @@ +# 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. + +from numbers import Number +import re + +from cassandra.metadata import protect_value +from cassandra.util import Point, LineString, Polygon +from geomet import wkt + + +def _raise_parse_error(msg): + """ + We need to throw ParseError which is defined in copyutil + since copyutil imports this file as part of it's initialization though, + importing it at the top of this file would create an import loop, so + we just import it locally when/if we need it + """ + from cqlshlib import copyutil + raise copyutil.ParseError(msg) + + +def _validate_point(val, raw): + if not all([isinstance(v, Number) for v in val]): + _raise_parse_error("Got non-numeric value in {}".format(raw)) + + if len(val) != 2: + _raise_parse_error("Got point with {} coordinates in: {}".format(len(val), raw)) + + +def _get_coords(val, expected_type): + geojson = wkt.loads(val) + gjtype = geojson.get('type') + if gjtype != expected_type: + _raise_parse_error("Expected {} type, but got {} type for {}".format(expected_type, gjtype, val)) + + return geojson.get("coordinates", []) + + +def _convert_point(val): + coords = _get_coords(val, 'Point') + _validate_point(coords, val) + point = Point(*coords) + return point + + +def _convert_linestring(val): + points = _get_coords(val, 'LineString') + for xy in points: + _validate_point(xy, val) + linestring = LineString(points) + return linestring + + +def _convert_polygon(val): + rings = _get_coords(val, 'Polygon') + if len(rings) == 0: + return Polygon([]) + + for ring in rings: + for point in ring: + _validate_point(point, val) + + polygon = Polygon(exterior=rings[0], interiors=rings[1:]) + return polygon + + +def _patch_get_converters(klass): + """ + patches the get converters method to convert WKT to dse.util.{Point, LineString, Polygon} + when using prepared statements to batch load + """ + original_method = klass._get_converter + + def new_method(self, cql_type): + if cql_type.typename == 'PointType': + return _convert_point + elif cql_type.typename == 'LineStringType': + return _convert_linestring + elif cql_type.typename == 'PolygonType': + return _convert_polygon + else: + return original_method(self, cql_type) + klass._get_converter = new_method + + +def _patch_init(klass): + """ + patches the constructor method to also protect (quote) geotype values + when making queries with string literal values + """ + original_method = klass.__init__ + + def new_method(self, *args, **kwargs): + original_method(self, *args, **kwargs) + ptypes = zip(self.protectors, self.coltypes) + clean = lambda t: re.sub("[\\W]", "", t.split('.')[-1]) # discard java package names and ' characters + gtypes = {'PointType', 'LineStringType', 'PolygonType'} + self.protectors = [protect_value if clean(t) in gtypes else p for p, t in ptypes] + klass.__init__ = new_method + + +def patch_geotypes_import_conversion(klass): + """ + monkey patches cqlshlib.copyutil.ImportConversion to support geotypes + """ + _patch_get_converters(klass) + _patch_init(klass) diff --git a/pylib/cqlshlib/test/test_copyutil.py b/pylib/cqlshlib/test/test_copyutil.py index 18b167adbfcf..151fe8756fab 100644 --- a/pylib/cqlshlib/test/test_copyutil.py +++ b/pylib/cqlshlib/test/test_copyutil.py @@ -20,10 +20,14 @@ import unittest -from cassandra.metadata import MIN_LONG, Murmur3Token, TokenMap +from cassandra.metadata import MIN_LONG, Murmur3Token from cassandra.policies import SimpleConvictionPolicy from cassandra.pool import Host -from unittest.mock import Mock + +try: + from unittest.mock import Mock +except ImportError: + from mock import Mock from cqlshlib.copyutil import ExportTask @@ -44,7 +48,9 @@ def setUp(self): Host('10.0.0.2', SimpleConvictionPolicy, 9000), Host('10.0.0.3', SimpleConvictionPolicy, 9000), Host('10.0.0.4', SimpleConvictionPolicy, 9000) - ] + ] + for h in self.hosts: + h.host_id = h.address def mock_shell(self): """ @@ -75,42 +81,39 @@ def _test_get_ranges_murmur3_base(self, opts, expected_ranges): } # merge override options with standard options overridden_opts = dict(self.opts) - for k,v in opts.items(): + for k, v in opts.items(): overridden_opts[k] = v export_task = ExportTask(shell, self.ks, self.table, self.columns, self.fname, overridden_opts, self.protocol_version, self.config_file) - assert export_task.get_ranges() == expected_ranges - - def test_get_ranges_murmur3(self): - """ - Test behavior of ExportTask internal get_ranges function - """ + export_ranges = export_task.get_ranges() + assert export_ranges == expected_ranges,\ + "Expected: {e}\n Actual:{a}".format(e=expected_ranges, a=export_ranges) + def test_murmur3_get_ranges_invalid_input(self): # return empty dict and print error if begin_token < min_token self._test_get_ranges_murmur3_base({'begintoken': MIN_LONG - 1}, {}) - - # return empty dict and print error if begin_token < min_token self._test_get_ranges_murmur3_base({'begintoken': 1, 'endtoken': -1}, {}) - # simple case of a single range - expected_ranges = {(1,2): {'hosts': ('10.0.0.4', '10.0.0.1', '10.0.0.2'), 'attempts': 0, 'rows': 0, 'workerno': -1}} + def test_get_ranges_murmur3_single_range(self): + expected_ranges = {(1, 2): {'hosts': ('10.0.0.4', '10.0.0.1', '10.0.0.2'), 'attempts': 0, 'rows': 0, 'workerno': -1}} self._test_get_ranges_murmur3_base({'begintoken': 1, 'endtoken': 2}, expected_ranges) - # simple case of two contiguous ranges + def test_get_ranges_murmur3_two_continuous_ranges(self): expected_ranges = { - (-4611686018427387903,0): {'hosts': ('10.0.0.3', '10.0.0.4', '10.0.0.1'), 'attempts': 0, 'rows': 0, 'workerno': -1}, - (0,1): {'hosts': ('10.0.0.4', '10.0.0.1', '10.0.0.2'), 'attempts': 0, 'rows': 0, 'workerno': -1} + (-4611686018427387903, 0): {'hosts': ('10.0.0.3', '10.0.0.4', '10.0.0.1'), 'attempts': 0, 'rows': 0, 'workerno': -1}, + (0, 1): {'hosts': ('10.0.0.4', '10.0.0.1', '10.0.0.2'), 'attempts': 0, 'rows': 0, 'workerno': -1} } self._test_get_ranges_murmur3_base({'begintoken': -4611686018427387903, 'endtoken': 1}, expected_ranges) + def test_get_ranges_murmur3_begin_token_only(self): # specify a begintoken only (endtoken defaults to None) expected_ranges = { - (4611686018427387905,None): {'hosts': ('10.0.0.1', '10.0.0.2', '10.0.0.3'), 'attempts': 0, 'rows': 0, 'workerno': -1} + (4611686018427387905, None): {'hosts': ('10.0.0.1', '10.0.0.2', '10.0.0.3'), 'attempts': 0, 'rows': 0, 'workerno': -1} } self._test_get_ranges_murmur3_base({'begintoken': 4611686018427387905}, expected_ranges) + def test_get_ranges_murmur3_end_token_only(self): # specify an endtoken only (begintoken defaults to None) expected_ranges = { (None, MIN_LONG + 1): {'hosts': ('10.0.0.2', '10.0.0.3', '10.0.0.4'), 'attempts': 0, 'rows': 0, 'workerno': -1} } self._test_get_ranges_murmur3_base({'endtoken': MIN_LONG + 1}, expected_ranges) - diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py index 5678def303c0..116a004ba9f7 100644 --- a/pylib/cqlshlib/test/test_cqlsh_output.py +++ b/pylib/cqlshlib/test/test_cqlsh_output.py @@ -665,6 +665,7 @@ def test_describe_columnfamily_output(self): AND comment = '' AND compaction = {'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32', 'min_threshold': '4'} AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'} + AND memtable = {} AND crc_check_chance = 1.0 AND default_time_to_live = 0 AND extensions = {} diff --git a/relocate-dependencies.pom b/relocate-dependencies.pom index 07728dd405ae..d7c9150dda7d 100644 --- a/relocate-dependencies.pom +++ b/relocate-dependencies.pom @@ -36,7 +36,7 @@ UTF-8UTF-8 - 4.12 + 4.13${java.version}${java.version}4.0.0-SNAPSHOT diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g index c1f6f125abc3..ba1e15a29d8d 100644 --- a/src/antlr/Lexer.g +++ b/src/antlr/Lexer.g @@ -216,6 +216,14 @@ K_DEFAULT: D E F A U L T; K_UNSET: U N S E T; K_LIKE: L I K E; +K_EDGE: E D G E; +K_VERTEX: V E R T E X; +K_LABEL: L A B E L; + +K_DROPPED: D R O P P E D; +K_COLUMN: C O L U M N; +K_RECORD: R E C O R D; + // Case-insensitive alpha characters fragment A: ('a'|'A'); fragment B: ('b'|'B'); diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index b3ba7b34c04d..cc386e178403 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -443,14 +443,24 @@ sident returns [Selectable.RawIdentifier id] whereClause returns [WhereClause.Builder clause] @init{ $clause = new WhereClause.Builder(); } - : relationOrExpression[$clause] (K_AND relationOrExpression[$clause])* + : expression[$clause] ; -relationOrExpression [WhereClause.Builder clause] - : relation[$clause] +expression [WhereClause.Builder clause] + : primaryExpression[$clause] (booleanOp[$clause] primaryExpression[$clause])* + ; + +primaryExpression [WhereClause.Builder clause] + : '(' {clause.startEnclosure(); } expression[$clause] ')' { clause.endEnclosure(); } + | relation[$clause] | customIndexExpression[$clause] ; +booleanOp [WhereClause.Builder clause] + : op=K_AND { clause.setCurrentOperator(op.getText()); } + | op=K_OR { clause.setCurrentOperator(op.getText()); } + ; + customIndexExpression [WhereClause.Builder clause] @init{QualifiedName name = new QualifiedName();} : 'expr(' idxName[name] ',' t=term ')' { clause.add(new CustomIndexExpression(name, t));} @@ -794,9 +804,21 @@ tablePartitionKey[CreateTableStatement.Raw stmt] ; tableProperty[CreateTableStatement.Raw stmt] + @init { boolean isStatic = false; } : property[stmt.attrs] | K_COMPACT K_STORAGE { $stmt.setCompactStorage(); } | K_CLUSTERING K_ORDER K_BY '(' tableClusteringOrder[stmt] (',' tableClusteringOrder[stmt])* ')' + | K_VERTEX K_LABEL ( noncol_ident )? {stmt.attrs.addProperty("dse_vertex_label_property", "vertex");} + | K_EDGE K_LABEL ( noncol_ident ) ? + K_FROM noncol_ident '(' ident (',' ident)* ')' + K_TO noncol_ident '(' ident (',' ident)* ')' + {stmt.attrs.addProperty("dse_edge_label_property", "edge");} + | K_DROPPED K_COLUMN K_RECORD + k=ident v=comparatorType (K_STATIC {isStatic = true;})? + K_USING K_TIMESTAMP t=INTEGER + { + stmt.attrs.addDroppedColumnRecord(k, v, isStatic, Long.parseLong($t.text)); + } ; tableClusteringOrder[CreateTableStatement.Raw stmt] @@ -942,7 +964,7 @@ alterTableStatement returns [AlterTableStatement.Raw stmt] | K_DROP ( id=ident { $stmt.drop(id); } | ('(' id1=ident { $stmt.drop(id1); } ( ',' idn=ident { $stmt.drop(idn); } )* ')') ) - ( K_USING K_TIMESTAMP t=INTEGER { $stmt.timestamp(Long.parseLong(Constants.Literal.integer($t.text).getText())); } )? + ( K_USING K_TIMESTAMP t=INTEGER { $stmt.dropTimestamp(Long.parseLong($t.text)); } )? | K_RENAME id1=ident K_TO toId1=ident { $stmt.rename(id1, toId1); } ( K_AND idn=ident K_TO toIdn=ident { $stmt.rename(idn, toIdn); } )* @@ -1701,7 +1723,6 @@ relation[WhereClause.Builder clauses] | type=relationType tupleMarker=markerForTuple /* (a, b, c) >= ? */ { $clauses.add(MultiColumnRelation.createNonInRelation(ids, type, tupleMarker)); } ) - | '(' relation[$clauses] ')' ; containsOperator returns [Operator o] @@ -1900,5 +1921,11 @@ basic_unreserved_keyword returns [String str] | K_MBEANS | K_REPLACE | K_UNSET + | K_EDGE + | K_VERTEX + | K_LABEL + | K_DROPPED + | K_COLUMN + | K_RECORD ) { $str = $k.text; } ; diff --git a/src/java/org/apache/cassandra/auth/FunctionResource.java b/src/java/org/apache/cassandra/auth/FunctionResource.java index 61c6a2966694..d47c019cf543 100644 --- a/src/java/org/apache/cassandra/auth/FunctionResource.java +++ b/src/java/org/apache/cassandra/auth/FunctionResource.java @@ -18,6 +18,7 @@ package org.apache.cassandra.auth; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.Set; @@ -187,7 +188,7 @@ public static FunctionResource fromName(String name) return keyspace(parts[1]); String[] nameAndArgs = StringUtils.split(parts[2], "[|]"); - return function(parts[1], nameAndArgs[0], argsListFromString(nameAndArgs[1])); + return function(parts[1], nameAndArgs[0], nameAndArgs.length > 1 ? argsListFromString(nameAndArgs[1]) : Collections.emptyList()); } /** diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java index 65ed71ec6459..0ecf9e516312 100644 --- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java +++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java @@ -43,6 +43,7 @@ import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.PageSize; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.ConsistencyLevel; @@ -222,7 +223,7 @@ private void replayFailedBatches() String query = String.format("SELECT id, mutations, version FROM %s.%s WHERE token(id) > token(?) AND token(id) <= token(?)", SchemaConstants.SYSTEM_KEYSPACE_NAME, SystemKeyspace.BATCHES); - UntypedResultSet batches = executeInternalWithPaging(query, pageSize, lastReplayedUuid, limitUuid); + UntypedResultSet batches = executeInternalWithPaging(query, PageSize.inRows(pageSize), lastReplayedUuid, limitUuid); processBatchlogEntries(batches, pageSize, rateLimiter); lastReplayedUuid = limitUuid; logger.trace("Finished replayFailedBatches"); diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java index 34f056a3c5eb..4c0930fcad8d 100644 --- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java +++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java @@ -34,16 +34,15 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.db.compaction.AbstractTableOperation; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.compaction.CompactionInfo; import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.compaction.OperationType; -import org.apache.cassandra.db.compaction.CompactionInfo.Unit; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.util.*; import org.apache.cassandra.io.util.CorruptFileException; @@ -285,10 +284,10 @@ public Future submitWrite(int keysToSave) return CompactionManager.instance.submitCacheWrite(getWriter(keysToSave)); } - public class Writer extends CompactionInfo.Holder + public class Writer extends AbstractTableOperation { private final Iterator keyIterator; - private final CompactionInfo info; + private final OperationProgress info; private long keysWritten; private final long keysEstimate; @@ -316,12 +315,12 @@ else if (cacheType == CacheService.CacheType.COUNTER_CACHE) else type = OperationType.UNKNOWN; - info = CompactionInfo.withoutSSTables(TableMetadata.minimal(SchemaConstants.SYSTEM_KEYSPACE_NAME, cacheType.toString()), - type, - 0, - keysEstimate, - Unit.KEYS, - UUIDGen.getTimeUUID()); + info = OperationProgress.withoutSSTables(TableMetadata.minimal(SchemaConstants.SYSTEM_KEYSPACE_NAME, cacheType.toString()), + type, + 0, + keysEstimate, + Unit.KEYS, + UUIDGen.getTimeUUID()); } public CacheService.CacheType cacheType() @@ -329,7 +328,7 @@ public CacheService.CacheType cacheType() return cacheType; } - public CompactionInfo getCompactionInfo() + public OperationProgress getProgress() { // keyset can change in size, thus total can too // TODO need to check for this one... was: info.forProgress(keysWritten, Math.max(keysWritten, keys.size())); diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index c53810ac4806..2b53552f3594 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -28,6 +28,9 @@ import com.google.common.collect.Iterables; import com.google.common.util.concurrent.MoreExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.github.benmanes.caffeine.cache.*; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.sstable.CorruptSSTableException; @@ -39,6 +42,8 @@ public class ChunkCache implements CacheLoader, RemovalListener, CacheSize { + private final static Logger logger = LoggerFactory.getLogger(ChunkCache.class); + public static final int RESERVED_POOL_SPACE_IN_MB = 32; public static final long cacheSize = 1024L * 1024L * Math.max(0, DatabaseDescriptor.getFileCacheSizeInMB() - RESERVED_POOL_SPACE_IN_MB); public static final boolean roundUp = DatabaseDescriptor.getFileCacheRoundUp(); @@ -226,15 +231,28 @@ public CachingRebufferer(ChunkReader file) @Override public Buffer rebuffer(long position) { + int spin = 0; try { long pageAlignedPos = position & alignmentMask; Buffer buf; - do - buf = cache.get(new Key(source, pageAlignedPos)).reference(); - while (buf == null); - - return buf; + Key key = new Key(source, pageAlignedPos); + while (true) + { + buf = cache.get(key).reference(); + if (buf != null) + return buf; + + if (++spin == 1000) + { + String msg = String.format("Could not acquire a reference to for %s after 1000 attempts. " + + "This is likely due to the chunk cache being too small for the " + + "number of concurrently running requests.", key); + throw new RuntimeException(msg); + // Note: this might also be caused by reference counting errors, especially double release of + // chunks. + } + } } catch (Throwable t) { diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 4a4002e738be..4919f345b869 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -36,6 +36,7 @@ import org.apache.cassandra.audit.AuditLogOptions; import org.apache.cassandra.fql.FullQueryLoggerOptions; import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.guardrails.GuardrailsConfig; /** * A class that contains configuration properties for the cassandra node it runs within. @@ -126,6 +127,7 @@ public class Config public Integer memtable_heap_space_in_mb; public Integer memtable_offheap_space_in_mb; public Float memtable_cleanup_threshold = null; + public Map memtable = null; // Limit the maximum depth of repair session merkle trees @Deprecated @@ -212,12 +214,28 @@ public class Config /* if the size of columns or super-columns are more than this, indexing will kick in */ public int column_index_size_in_kb = 64; public volatile int column_index_cache_size_in_kb = 2; - public volatile int batch_size_warn_threshold_in_kb = 5; - public volatile int batch_size_fail_threshold_in_kb = 50; - public Integer unlogged_batch_across_partitions_warn_threshold = 10; + /** + * @deprecated Migrated to 'guardrails.batch_size_warn_threshold_in_kb' + */ + @Deprecated + public int batch_size_warn_threshold_in_kb = 0; + /** + * @deprecated Migrated to 'guardrails.batch_size_fail_threshold_in_kb' + */ + @Deprecated + public int batch_size_fail_threshold_in_kb = 0; + /** + * @deprecated Migrated to 'guardrails.unlogged_batch_across_partitions_warn_threshold' + */ + @Deprecated + public Integer unlogged_batch_across_partitions_warn_threshold = 0; public volatile Integer concurrent_compactors; public volatile int compaction_throughput_mb_per_sec = 16; - public volatile int compaction_large_partition_warning_threshold_mb = 100; + /** + * @deprecated Migrated to 'guardrails.compaction_large_partition_warning_threshold_mb' + */ + @Deprecated + public int compaction_large_partition_warning_threshold_mb = 0; public int min_free_space_per_drive_in_mb = 50; public volatile int concurrent_materialized_view_builders = 1; @@ -290,7 +308,7 @@ public class Config public ParameterizedClass hints_compression; public volatile boolean incremental_backups = false; - public boolean trickle_fsync = false; + public boolean trickle_fsync = true; public int trickle_fsync_interval_in_kb = 10240; public volatile int sstable_preemptive_open_interval_in_mb = 50; @@ -340,10 +358,18 @@ public class Config public boolean inter_dc_tcp_nodelay = true; - public MemtableAllocationType memtable_allocation_type = MemtableAllocationType.heap_buffers; + public MemtableAllocationType memtable_allocation_type = MemtableAllocationType.offheap_objects; - public volatile int tombstone_warn_threshold = 1000; - public volatile int tombstone_failure_threshold = 100000; + /** + * @deprecated Migrated to 'guardrails.tombstone_warn_threshold' + */ + @Deprecated + public int tombstone_warn_threshold = 0; + /** + * @deprecated Migrated to 'guardrails.tombstone_failure_threshold' + */ + @Deprecated + public int tombstone_failure_threshold = 0; public final ReplicaFilteringProtectionOptions replica_filtering_protection = new ReplicaFilteringProtectionOptions(); @@ -504,6 +530,10 @@ public class Config */ public volatile int validation_preview_purge_head_start_in_sec = 60 * 60; + public boolean emulate_dbaas_defaults = false; + + public GuardrailsConfig guardrails = new GuardrailsConfig(); + /** * The intial capacity for creating RangeTombstoneList. */ @@ -513,6 +543,10 @@ public class Config */ public volatile double range_tombstone_list_growth_factor = 1.5; + public StorageAttachedIndexOptions sai_options = new StorageAttachedIndexOptions(); + + public volatile int aggregation_subpage_size_in_kb = 2048; + /** * @deprecated migrate to {@link DatabaseDescriptor#isClientInitialized()} */ diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 79057b91eb7a..9e5ffd4f1429 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.net.*; import java.nio.file.FileStore; +import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.Paths; @@ -31,7 +32,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.HashMultiset; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multiset; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.RateLimiter; @@ -40,6 +43,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.audit.AuditLogOptions; +import org.apache.cassandra.cql3.PageSize; import org.apache.cassandra.fql.FullQueryLoggerOptions; import org.apache.cassandra.auth.AllowAllInternodeAuthenticator; import org.apache.cassandra.auth.AuthConfig; @@ -56,6 +60,7 @@ import org.apache.cassandra.db.commitlog.CommitLogSegmentManagerStandard; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.guardrails.GuardrailsConfig; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.util.DiskOptimizationStrategy; import org.apache.cassandra.io.util.FileUtils; @@ -176,6 +181,10 @@ public static void daemonInitialization(Supplier config) throws Configur setConfig(config.get()); applyAll(); + + createAllDirectories(); + applyGuardrails(); // requires created directories + AuthConfig.applyAuth(); } @@ -364,6 +373,13 @@ private static void applyAll() throws ConfigurationException applySslContext(); } + private static void applyGuardrails() + { + conf.guardrails.applyConfig(); + conf.guardrails.validate(); + getGuardrailsConfig().validateAfterDataDirectoriesExist(); + } + private static void applySimpleConfig() { //Doing this first before all other things in case other pieces of config want to construct @@ -411,7 +427,7 @@ else if (!Double.isNaN(conf.commitlog_sync_batch_window_in_ms)) /* evaluate the DiskAccessMode Config directive, which also affects indexAccessMode selection */ if (conf.disk_access_mode == Config.DiskAccessMode.auto) { - conf.disk_access_mode = hasLargeAddressSpace() ? Config.DiskAccessMode.mmap : Config.DiskAccessMode.standard; + conf.disk_access_mode = Config.DiskAccessMode.standard; indexAccessMode = conf.disk_access_mode; logger.info("DiskAccessMode 'auto' determined to be {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode); } @@ -595,7 +611,7 @@ else if (conf.repair_session_space_in_mb > (int) (Runtime.getRuntime().maxMemory } } - logger.info("cdc_enabled is true. Starting casssandra node with Change-Data-Capture enabled."); + logger.info("cdc_enabled is true. Starting cassandra node with Change-Data-Capture enabled."); } if (conf.saved_caches_directory == null) @@ -755,6 +771,48 @@ else if (conf.repair_session_space_in_mb > (int) (Runtime.getRuntime().maxMemory if (conf.user_defined_function_fail_timeout < conf.user_defined_function_warn_timeout) throw new ConfigurationException("user_defined_function_warn_timeout must less than user_defined_function_fail_timeout", false); + if (conf.compaction_large_partition_warning_threshold_mb != 0) + { + logger.warn("Found deprecated property 'compaction_large_partition_warning_threshold_mb' in config - migrate to `guardrails.partition_size_warn_threshold_in_mb`. " + + "The value of 'guardrails.partition_size_warn_threshold_in_mb' is overwritten by 'compaction_large_partition_warning_threshold_mb'."); + getGuardrailsConfig().partition_size_warn_threshold_in_mb = conf.compaction_large_partition_warning_threshold_mb; + } + + if (conf.tombstone_failure_threshold != 0) + { + logger.warn("Found deprecated property 'tombstone_failure_threshold' in config - migrate to 'guardrails.tombstone_failure_threshold'. " + + "The value of 'guardrails.tombstone_failure_threshold' is overwritten by 'tombstone_failure_threshold'."); + getGuardrailsConfig().tombstone_failure_threshold = conf.tombstone_failure_threshold; + } + + if (conf.tombstone_warn_threshold != 0) + { + logger.warn("Found deprecated property 'tombstone_warn_threshold' in config - migrate to 'guardrails.tombstone_warn_threshold'. " + + "The value of 'guardrails.tombstone_warn_threshold' is overwritten by 'tombstone_warn_threshold'."); + getGuardrailsConfig().tombstone_warn_threshold = conf.tombstone_warn_threshold; + } + + if (conf.batch_size_fail_threshold_in_kb != 0) + { + logger.warn("Found deprecated property 'batch_size_fail_threshold_in_kb' in config - migrate to 'guardrails.batch_size_fail_threshold_in_kb'. " + + "The value of 'guardrails.batch_size_fail_threshold_in_kb' is overwritten by 'batch_size_fail_threshold_in_kb'."); + getGuardrailsConfig().batch_size_fail_threshold_in_kb = conf.batch_size_fail_threshold_in_kb; + } + + if (conf.batch_size_warn_threshold_in_kb != 0) + { + logger.warn("Found deprecated property 'batch_size_warn_threshold_in_kb' in config - migrate to 'guardrails.batch_size_warn_threshold_in_kb'. " + + "The value of 'guardrails.batch_size_warn_threshold_in_kb' is overwritten by 'batch_size_warn_threshold_in_kb'."); + getGuardrailsConfig().batch_size_warn_threshold_in_kb = conf.batch_size_warn_threshold_in_kb; + } + + if (conf.unlogged_batch_across_partitions_warn_threshold != 0) + { + logger.warn("Found deprecated property 'unlogged_batch_across_partitions_warn_threshold' in config - migrate to 'guardrails.unlogged_batch_across_partitions_warn_threshold'. " + + "The value of 'guardrails.unlogged_batch_across_partitions_warn_threshold' is overwritten by 'unlogged_batch_across_partitions_warn_threshold'."); + getGuardrailsConfig().unlogged_batch_across_partitions_warn_threshold = conf.unlogged_batch_across_partitions_warn_threshold; + } + if (conf.commitlog_segment_size_in_mb <= 0) throw new ConfigurationException("commitlog_segment_size_in_mb must be positive, but was " + conf.commitlog_segment_size_in_mb, false); @@ -838,6 +896,11 @@ else if (conf.max_value_size_in_mb >= 2048) } validateMaxConcurrentAutoUpgradeTasksConf(conf.max_concurrent_automatic_sstable_upgrades); + + if (conf.aggregation_subpage_size_in_kb < 1) + throw new ConfigurationException("aggregation_subpage_size_in_kb must be greater than 0"); + + setAggregationSubPageSize(getAggregationSubPageSize()); } @VisibleForTesting @@ -1483,42 +1546,6 @@ public static void setColumnIndexCacheSize(int val) conf.column_index_cache_size_in_kb = val; } - public static int getBatchSizeWarnThreshold() - { - return (int) ByteUnit.KIBI_BYTES.toBytes(conf.batch_size_warn_threshold_in_kb); - } - - public static int getBatchSizeWarnThresholdInKB() - { - return conf.batch_size_warn_threshold_in_kb; - } - - public static long getBatchSizeFailThreshold() - { - return ByteUnit.KIBI_BYTES.toBytes(conf.batch_size_fail_threshold_in_kb); - } - - public static int getBatchSizeFailThresholdInKB() - { - return conf.batch_size_fail_threshold_in_kb; - } - - public static int getUnloggedBatchAcrossPartitionsWarnThreshold() - { - return conf.unlogged_batch_across_partitions_warn_threshold; - } - - public static void setBatchSizeWarnThresholdInKB(int threshold) - { - checkValidForByteConversion(threshold, "batch_size_warn_threshold_in_kb", ByteUnit.KIBI_BYTES); - conf.batch_size_warn_threshold_in_kb = threshold; - } - - public static void setBatchSizeFailThresholdInKB(int threshold) - { - conf.batch_size_fail_threshold_in_kb = threshold; - } - public static Collection getInitialTokens() { return tokensFromString(System.getProperty(Config.PROPERTY_PREFIX + "initial_token", conf.initial_token)); @@ -1799,8 +1826,6 @@ public static void setCompactionThroughputMbPerSec(int value) conf.compaction_throughput_mb_per_sec = value; } - public static long getCompactionLargePartitionWarningThreshold() { return ByteUnit.MEBI_BYTES.toBytes(conf.compaction_large_partition_warning_threshold_mb); } - public static int getConcurrentValidations() { return conf.concurrent_validations; @@ -1908,6 +1933,50 @@ public static String[] getAllDataFileLocations() return ArrayUtils.addFirst(conf.data_file_directories, conf.local_system_data_file_directory); } + /** + * @return Minimum total space for all the data file directories in GB. + * 0 if fail to get the total space, or total space is under 1 GB. + */ + public static long getDataFileDirectoriesMinTotalSpaceInGB() + { + String[] dataDirectories = getAllDataFileLocations(); + if (dataDirectories.length == 0) + { + return 0L; + } + + Multiset fileStores = HashMultiset.create(); + for (String dir : dataDirectories) + { + try + { + fileStores.add(Files.getFileStore(new File(dir).toPath())); + } + catch (IOException ioe) + { + logger.warn("Unable to get FileStore of {}. {}", dir, ioe); + } + } + return getDataFileDirectoriesMinTotalSpaceInGB(fileStores); + } + + @VisibleForTesting + static long getDataFileDirectoriesMinTotalSpaceInGB(Multiset fileStores) + { + return fileStores.entrySet().stream().mapToLong(entry -> { + long totalSpace = 0L; + try + { + totalSpace = FileUtils.handleLargeFileSystem(entry.getElement().getTotalSpace()); + } + catch (IOException ioe) + { + logger.warn("Unable to get total space of {}. {}", entry.getElement(), ioe); + } + return (totalSpace >> 30) / entry.getCount(); + }).min().orElse(0L) * fileStores.size(); + } + public static String getCommitLogLocation() { return conf.commitlog_directory; @@ -1959,26 +2028,6 @@ public static int getMaxMutationSize() return (int) ByteUnit.KIBI_BYTES.toBytes(conf.max_mutation_size_in_kb); } - public static int getTombstoneWarnThreshold() - { - return conf.tombstone_warn_threshold; - } - - public static void setTombstoneWarnThreshold(int threshold) - { - conf.tombstone_warn_threshold = threshold; - } - - public static int getTombstoneFailureThreshold() - { - return conf.tombstone_failure_threshold; - } - - public static void setTombstoneFailureThreshold(int threshold) - { - conf.tombstone_failure_threshold = threshold; - } - public static int getCachedReplicaRowsWarnThreshold() { return conf.replica_filtering_protection.cached_rows_warn_threshold; @@ -2849,6 +2898,11 @@ public static Float getMemtableCleanupThreshold() return conf.memtable_cleanup_threshold; } + public static Map getMemtableOptions() + { + return conf.memtable; + } + public static int getIndexSummaryResizeIntervalInMinutes() { return conf.index_summary_resize_interval_in_minutes; @@ -3375,4 +3429,52 @@ public static void setConsecutiveMessageErrorsThreshold(int value) { conf.consecutive_message_errors_threshold = value; } + + public static int getSAISegmentWriteBufferSpace() + { + return conf.sai_options.segment_write_buffer_space_mb; + } + + public static void setSAISegmentWriteBufferSpace(int bufferSpace) + { + conf.sai_options.segment_write_buffer_space_mb = bufferSpace; + } + + public static double getSAIZeroCopyUsedThreshold() + { + return conf.sai_options.zerocopy_used_threshold; + } + + public static void setSAIZeroCopyUsedThreshold(double threshold) + { + conf.sai_options.zerocopy_used_threshold = threshold; + } + + public static GuardrailsConfig getGuardrailsConfig() + { + return conf.guardrails; + } + + @VisibleForTesting + public static boolean setEmulateDbaasDefaults(boolean dbaasDefaults) + { + return conf.emulate_dbaas_defaults = dbaasDefaults; + } + + public static boolean isEmulateDbaasDefaults() + { + return conf.emulate_dbaas_defaults; + } + + public static PageSize getAggregationSubPageSize() + { + return PageSize.inBytes(conf.aggregation_subpage_size_in_kb * 1024); + } + + public static void setAggregationSubPageSize(PageSize pageSize) + { + Preconditions.checkArgument(!pageSize.isDefined() || pageSize.getUnit() == PageSize.PageUnit.BYTES); + Preconditions.checkArgument(pageSize.bytes() >= 1024); + conf.aggregation_subpage_size_in_kb = pageSize.bytes() / 1024; + } } diff --git a/src/java/org/apache/cassandra/config/StorageAttachedIndexOptions.java b/src/java/org/apache/cassandra/config/StorageAttachedIndexOptions.java new file mode 100644 index 000000000000..84e8d8aee130 --- /dev/null +++ b/src/java/org/apache/cassandra/config/StorageAttachedIndexOptions.java @@ -0,0 +1,64 @@ +/* + * 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.config; + +import com.google.common.base.Objects; + +import org.apache.cassandra.exceptions.ConfigurationException; + +public class StorageAttachedIndexOptions +{ + public static final int DEFAULT_SEGMENT_BUFFER_MB = 1024; + public static final double DEFAULT_ZEROCOPY_USED_THRESHOLD = 0.3; + + private static final int MAXIMUM_SEGMENT_BUFFER_MB = 32768; + + public int segment_write_buffer_space_mb = DEFAULT_SEGMENT_BUFFER_MB; + public double zerocopy_used_threshold = DEFAULT_ZEROCOPY_USED_THRESHOLD; + + public void validate() + { + if ((segment_write_buffer_space_mb < 0) || (segment_write_buffer_space_mb > MAXIMUM_SEGMENT_BUFFER_MB)) + { + throw new ConfigurationException("Invalid value for segment_write_buffer_space_mb. " + + "Value must be a positive integer less than 32768"); + } + + if ((zerocopy_used_threshold < 0.0) || (zerocopy_used_threshold > 1.0)) + { + throw new ConfigurationException("Invalid value for zero_copy_used_threshold. " + + "Value must be between 0.0 and 1.0"); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + StorageAttachedIndexOptions that = (StorageAttachedIndexOptions) o; + return Objects.equal(segment_write_buffer_space_mb, that.segment_write_buffer_space_mb) && + Objects.equal(zerocopy_used_threshold, that.zerocopy_used_threshold); + } + + @Override + public int hashCode() + { + return Objects.hashCode(segment_write_buffer_space_mb, zerocopy_used_threshold); + } +} diff --git a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java index ac8f179fe43f..28178c7a32fe 100644 --- a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java +++ b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java @@ -69,9 +69,9 @@ public String getKeyspace() return wrapped.getKeyspace(); } - public ConsistencyLevel getSerialConsistency() + public ConsistencyLevel getSerialConsistency(QueryState state) { - return wrapped.getSerialConsistency(); + return wrapped.getSerialConsistency(state); } public List getQueryOrIdList() diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java index 5059104446e5..421b44684e6d 100644 --- a/src/java/org/apache/cassandra/cql3/CQL3Type.java +++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java @@ -196,7 +196,7 @@ public String toCQLLiteral(ByteBuffer buffer, ProtocolVersion version) StringBuilder target = new StringBuilder(); buffer = buffer.duplicate(); - int size = CollectionSerializer.readCollectionSize(buffer, version); + int size = CollectionSerializer.readCollectionSize(buffer, ByteBufferAccessor.instance, version); buffer.position(buffer.position() + CollectionSerializer.sizeOfCollectionSize(size, version)); switch (type.kind) diff --git a/src/java/org/apache/cassandra/cql3/CQLStatement.java b/src/java/org/apache/cassandra/cql3/CQLStatement.java index c34e27fb2eb4..efc6bcfd39b8 100644 --- a/src/java/org/apache/cassandra/cql3/CQLStatement.java +++ b/src/java/org/apache/cassandra/cql3/CQLStatement.java @@ -66,9 +66,9 @@ default Iterable getFunctions() /** * Perform additional validation required by the statment. To be overriden by subclasses if needed. * - * @param state the current client state + * @param state the current query state */ - public void validate(ClientState state); + public void validate(QueryState state); /** * Execute the statement and return the resulting result or null if there is no result. diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java index 1d94d697a50e..38029d1f33a2 100644 --- a/src/java/org/apache/cassandra/cql3/Lists.java +++ b/src/java/org/apache/cassandra/cql3/Lists.java @@ -29,6 +29,7 @@ import java.util.stream.StreamSupport; import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.ColumnMetadata; import com.google.common.annotations.VisibleForTesting; import org.apache.cassandra.cql3.functions.Function; @@ -128,8 +129,30 @@ public static String listToString(Iterable items, java.util.function.Func public static AbstractType getExactListTypeIfKnown(List items, java.util.function.Function> mapper) { - Optional> type = items.stream().map(mapper).filter(Objects::nonNull).findFirst(); - return type.isPresent() ? ListType.getInstance(type.get(), false) : null; + AbstractType type = getElementType(items, mapper); + return type != null ? ListType.getInstance(type, false) : null; + } + + protected static AbstractType getElementType(List items, + java.util.function.Function> mapper) + { + AbstractType type = null; + for (T item : items) + { + AbstractType itemType = mapper.apply(item); + if (itemType == null) + continue; + + if (type != null && !itemType.isCompatibleWith(type)) + { + if (type.isCompatibleWith(itemType)) + continue; + + throw new InvalidRequestException("Invalid collection literal: all selectors must have the same CQL type inside collection literals"); + } + type = itemType; + } + return type; } public static class Literal extends Term.Raw @@ -442,11 +465,14 @@ public void collectMarkerSpecification(VariableSpecifications boundNames) idx.collectMarkerSpecification(boundNames); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { // we should not get here for frozen lists assert column.type.isMultiCell() : "Attempted to set an individual element on a frozen list"; + Guardrails.readBeforeWriteListOperationsEnabled.ensureEnabled("Setting of list items by index requiring read before write", params.state); + ByteBuffer index = idx.bindAndGet(params.options); ByteBuffer value = t.bindAndGet(params.options); @@ -478,6 +504,7 @@ public Appender(ColumnMetadata column, Term t) super(column, t); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to append to a frozen list"; @@ -487,26 +514,40 @@ public void execute(DecoratedKey partitionKey, UpdateParameters params) throws I static void doAppend(Term.Terminal value, ColumnMetadata column, UpdateParameters params) throws InvalidRequestException { - if (column.type.isMultiCell()) + if (value == null) { + // for frozen lists, we're overwriting the whole cell value + if (!column.type.isMultiCell()) + params.addTombstone(column); + // If we append null, do nothing. Note that for Setter, we've // already removed the previous value so we're good here too - if (value == null) - return; + return; + } + + List elements = ((Value) value).elements; - for (ByteBuffer buffer : ((Value) value).elements) + if (column.type.isMultiCell()) + { + // Guardrails about collection size are only checked for the added elements without considering + // already existent elements. This is done so to avoid read-before-write, having additional checks + // during SSTable write. + Guardrails.itemsPerCollection.guard(elements.size(), column.name.toString(), false, params.state); + + int dataSize = 0; + for (ByteBuffer buffer : elements) { ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()); - params.addCell(column, CellPath.create(uuid), buffer); + Cell cell = params.addCell(column, CellPath.create(uuid), buffer); + dataSize += cell.dataSize(); } + Guardrails.collectionSize.guard(dataSize, column.name.toString(), false, params.state); } else { - // for frozen lists, we're overwriting the whole cell value - if (value == null) - params.addTombstone(column); - else - params.addCell(column, value.get(ProtocolVersion.CURRENT)); + Guardrails.itemsPerCollection.guard(elements.size(), column.name.toString(), false, params.state); + Cell cell = params.addCell(column, value.get(ProtocolVersion.CURRENT)); + Guardrails.collectionSize.guard(cell.dataSize(), column.name.toString(), false, params.state); } } } @@ -528,10 +569,16 @@ public void execute(DecoratedKey partitionKey, UpdateParameters params) throws I List toAdd = ((Value) value).elements; final int totalCount = toAdd.size(); + // Guardrails about collection size are only checked for the added elements without considering + // already existent elements. This is done so to avoid read-before-write, having additional checks + // during SSTable write. + Guardrails.itemsPerCollection.guard(totalCount, column.name.toString(), false, params.state); + // we have to obey MAX_NANOS per batch - in the unlikely event a client has decided to prepend a list with // an insane number of entries. PrecisionTime pt = null; int remainingInBatch = 0; + int dataSize = 0; for (int i = totalCount - 1; i >= 0; i--) { if (remainingInBatch == 0) @@ -542,8 +589,10 @@ public void execute(DecoratedKey partitionKey, UpdateParameters params) throws I } ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(pt.millis, (pt.nanos + remainingInBatch--))); - params.addCell(column, CellPath.create(uuid), toAdd.get(i)); + Cell cell = params.addCell(column, CellPath.create(uuid), toAdd.get(i)); + dataSize += cell.dataSize(); } + Guardrails.collectionSize.guard(dataSize, column.name.toString(), false, params.state); } } @@ -560,10 +609,13 @@ public boolean requiresRead() return true; } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to delete from a frozen list"; + Guardrails.readBeforeWriteListOperationsEnabled.ensureEnabled("Removal of list items requiring read before write", params.state); + // We want to call bind before possibly returning to reject queries where the value provided is not a list. Term.Terminal value = t.bind(params.options); @@ -598,9 +650,12 @@ public boolean requiresRead() return true; } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to delete an item by index from a frozen list"; + + Guardrails.readBeforeWriteListOperationsEnabled.ensureEnabled("Removal of list items by index requiring read before write", params.state); Term.Terminal index = t.bind(params.options); if (index == null) throw new InvalidRequestException("Invalid null value for list index"); diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java index 6e7e07b57601..4eb073ccb84f 100644 --- a/src/java/org/apache/cassandra/cql3/Maps.java +++ b/src/java/org/apache/cassandra/cql3/Maps.java @@ -23,6 +23,7 @@ import java.util.*; import java.util.stream.Collectors; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.db.DecoratedKey; @@ -134,16 +135,31 @@ public static AbstractType getExactMapTypeIfKnown(List> entrie AbstractType valueType = null; for (Pair entry : entries) { - if (keyType == null) - keyType = mapper.apply(entry.left); - if (valueType == null) - valueType = mapper.apply(entry.right); - if (keyType != null && valueType != null) - return MapType.getInstance(keyType, valueType, false); + keyType = selectType(keyType, mapper.apply(entry.left)); + valueType = selectType(valueType, mapper.apply(entry.right)); } + + if (keyType != null && valueType != null) + return MapType.getInstance(keyType, valueType, false); + return null; } + private static AbstractType selectType(AbstractType type, AbstractType otherType) + { + if (otherType == null) + return type; + + if (type != null && !otherType.isCompatibleWith(type)) + { + if (type.isCompatibleWith(otherType)) + return type; + + throw new InvalidRequestException("Invalid collection literal: all selectors must have the same CQL type inside collection literals"); + } + return otherType; + } + public static class Literal extends Term.Raw { public final List> entries; @@ -351,6 +367,7 @@ public Setter(ColumnMetadata column, Term t) super(column, t); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { Term.Terminal value = t.bind(params.options); @@ -381,6 +398,7 @@ public void collectMarkerSpecification(VariableSpecifications boundNames) k.collectMarkerSpecification(boundNames); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to set a value for a single key on a frozen map"; @@ -411,6 +429,7 @@ public Putter(ColumnMetadata column, Term t) super(column, t); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to add items to a frozen map"; @@ -421,22 +440,37 @@ public void execute(DecoratedKey partitionKey, UpdateParameters params) throws I static void doPut(Term.Terminal value, ColumnMetadata column, UpdateParameters params) throws InvalidRequestException { + if (value == null) + { + // for frozen maps, we're overwriting the whole cell + if (!column.type.isMultiCell()) + params.addTombstone(column); + + return; + } + + Map elements = ((Value) value).map; + if (column.type.isMultiCell()) { - if (value == null) - return; + // Guardrails about collection size are only checked for the added elements without considering + // already existent elements. This is done so to avoid read-before-write, having additional checks + // during SSTable write. + Guardrails.itemsPerCollection.guard(elements.size(), column.name.toString(), false, params.state); - Map elements = ((Value) value).map; + int dataSize = 0; for (Map.Entry entry : elements.entrySet()) - params.addCell(column, CellPath.create(entry.getKey()), entry.getValue()); + { + Cell cell = params.addCell(column, CellPath.create(entry.getKey()), entry.getValue()); + dataSize += cell.dataSize(); + } + Guardrails.collectionSize.guard(dataSize, column.name.toString(), false, params.state); } else { - // for frozen maps, we're overwriting the whole cell - if (value == null) - params.addTombstone(column); - else - params.addCell(column, value.get(ProtocolVersion.CURRENT)); + Guardrails.itemsPerCollection.guard(elements.size(), column.name.toString(), false, params.state); + Cell cell = params.addCell(column, value.get(ProtocolVersion.CURRENT)); + Guardrails.collectionSize.guard(cell.dataSize(), column.name.toString(), false, params.state); } } } @@ -448,6 +482,7 @@ public DiscarderByKey(ColumnMetadata column, Term k) super(column, k); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to delete a single key in a frozen map"; diff --git a/src/java/org/apache/cassandra/cql3/Operator.java b/src/java/org/apache/cassandra/cql3/Operator.java index 1acedeeeab90..7a900b8e6c26 100644 --- a/src/java/org/apache/cassandra/cql3/Operator.java +++ b/src/java/org/apache/cassandra/cql3/Operator.java @@ -316,6 +316,15 @@ public int serializedSize() return 4; } + /** + * Checks if this operator is a like operator. + * @return {@code true} if this operator is a like operator, {@code false} otherwise. + */ + public boolean isLike() + { + return this == LIKE_PREFIX || this == LIKE_CONTAINS || this == LIKE_SUFFIX || this == LIKE_MATCHES; + } + /** * Checks if this operator is a slice operator. * @return {@code true} if this operator is a slice operator, {@code false} otherwise. diff --git a/src/java/org/apache/cassandra/cql3/PageSize.java b/src/java/org/apache/cassandra/cql3/PageSize.java new file mode 100644 index 000000000000..2248610dec59 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/PageSize.java @@ -0,0 +1,153 @@ +/* + * 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.util.Objects; + +import com.google.common.base.Preconditions; + +public class PageSize +{ + public static final int NO_LIMIT = Integer.MAX_VALUE; + + public static final PageSize NONE = new PageSize(NO_LIMIT, PageUnit.ROWS); + + public enum PageUnit + { + ROWS, BYTES + } + + private final int size; + private final PageUnit unit; + + public PageSize(int size, PageUnit unit) + { + Preconditions.checkArgument(size >= 0); + Preconditions.checkNotNull(unit); + this.size = size; + this.unit = unit; + } + + public int getSize() + { + return size; + } + + public PageUnit getUnit() + { + return unit; + } + + public int bytes() + { + return unit == PageUnit.BYTES ? size : NO_LIMIT; + } + + public int rows() + { + return unit == PageUnit.ROWS ? size : NO_LIMIT; + } + + /** + * Creates a page size representing {@code count} rows. + * + * @throws IllegalArgumentException if the size is not strictly positive. + */ + public static PageSize inRows(int rowsCount) + { + return new PageSize(rowsCount, PageUnit.ROWS); + } + + /** + * Creates a page size representing {@code size} bytes. + * + * @throws IllegalArgumentException if the size is not strictly positive. + */ + public static PageSize inBytes(int bytesCount) + { + return new PageSize(bytesCount, PageUnit.BYTES); + } + + /** + * Returns the minimum number of rows for the given number and the number of rows represented by this page size. + * If this page size is defined in bytes or undefined, it will just return the provided number of rows. + */ + public int minRowsCount(int rowsCount) + { + return unit == PageUnit.ROWS ? Math.min(rowsCount, size) : rowsCount; + } + + /** + * Returns the minimum number of bytes for the given number and the number of bytes represented by this page size. + * If this page size is defined in rows or undefined, it will just return the provided number of bytes. + */ + public int minBytesCount(int bytesCount) + { + return unit == PageUnit.BYTES ? Math.min(bytesCount, size) : bytesCount; + } + + public boolean isDefined() + { + return size < NO_LIMIT; + } + + public PageSize withDecreasedRows(int rowsCount) { + return unit == PageUnit.ROWS && size != NO_LIMIT + ? inRows(Math.max(0, size - rowsCount)) + : this; + } + + public PageSize withDecreasedBytes(int bytesCount) { + return unit == PageUnit.BYTES && size != NO_LIMIT + ? inBytes(Math.max(0, size - bytesCount)) + : this; + } + + /** + * Assuming we went through the provided number of rows/bytes, it returns whether the page is completed. + * It will always return {@code false} if the page size is undefined (unlimited). + */ + public boolean isCompleted(int count, PageUnit unit) + { + return this.unit == unit && this.size <= count; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + PageSize pageSize = (PageSize) o; + return size == pageSize.size && unit == pageSize.unit; + } + + @Override + public int hashCode() + { + return Objects.hash(size, unit); + } + + @Override + public String toString() + { + if (size == NO_LIMIT) + return "unlimited"; + else + return size + " " + unit.name().toLowerCase(); + } +} diff --git a/src/java/org/apache/cassandra/cql3/QueryOptions.java b/src/java/org/apache/cassandra/cql3/QueryOptions.java index d3b1a03cca9e..8f077e0d2536 100644 --- a/src/java/org/apache/cassandra/cql3/QueryOptions.java +++ b/src/java/org/apache/cassandra/cql3/QueryOptions.java @@ -19,15 +19,17 @@ import java.nio.ByteBuffer; import java.util.*; +import javax.annotation.Nullable; import com.google.common.collect.ImmutableList; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import io.netty.buffer.ByteBuf; - -import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.pager.PagingState; import org.apache.cassandra.transport.CBCodec; @@ -35,8 +37,6 @@ import org.apache.cassandra.transport.ProtocolException; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.Pair; -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; /** * Options for a query. @@ -72,7 +72,7 @@ public static QueryOptions forProtocolVersion(ProtocolVersion protocolVersion) public static QueryOptions create(ConsistencyLevel consistency, List values, boolean skipMetadata, - int pageSize, + PageSize pageSize, PagingState pagingState, ConsistencyLevel serialConsistency, ProtocolVersion version, @@ -84,7 +84,7 @@ public static QueryOptions create(ConsistencyLevel consistency, public static QueryOptions create(ConsistencyLevel consistency, List values, boolean skipMetadata, - int pageSize, + PageSize pageSize, PagingState pagingState, ConsistencyLevel serialConsistency, ProtocolVersion version, @@ -175,8 +175,8 @@ public ImmutableList getColumnSpecifications() throw new UnsupportedOperationException(); } - /** The pageSize for this query. Will be {@code <= 0} if not relevant for the query. */ - public int getPageSize() + /** The pageSize for this query. Will be {@code <= 0} if not relevant for the query. */ + public PageSize getPageSize() { return getSpecificOptions().pageSize; } @@ -188,9 +188,10 @@ public PagingState getPagingState() } /** Serial consistency for conditional updates. */ - public ConsistencyLevel getSerialConsistency() + public ConsistencyLevel getSerialConsistency(@Nullable QueryState state) { - return getSpecificOptions().serialConsistency; + ConsistencyLevel cl = getSpecificOptions().serialConsistency; + return cl != null ? cl : ConsistencyLevel.defaultSerialConsistency(state); } public long getTimestamp(QueryState state) @@ -377,16 +378,16 @@ public List getValues() // Options that are likely to not be present in most queries static class SpecificOptions { - private static final SpecificOptions DEFAULT = new SpecificOptions(-1, null, null, Long.MIN_VALUE, null, Integer.MIN_VALUE); + private static final SpecificOptions DEFAULT = new SpecificOptions(PageSize.NONE, null, null, Long.MIN_VALUE, null, Integer.MIN_VALUE); - private final int pageSize; + private final PageSize pageSize; private final PagingState state; private final ConsistencyLevel serialConsistency; private final long timestamp; private final String keyspace; private final int nowInSeconds; - private SpecificOptions(int pageSize, + private SpecificOptions(PageSize pageSize, PagingState state, ConsistencyLevel serialConsistency, long timestamp, @@ -395,7 +396,7 @@ private SpecificOptions(int pageSize, { this.pageSize = pageSize; this.state = state; - this.serialConsistency = serialConsistency == null ? ConsistencyLevel.SERIAL : serialConsistency; + this.serialConsistency = serialConsistency; this.timestamp = timestamp; this.keyspace = keyspace; this.nowInSeconds = nowInSeconds; @@ -415,7 +416,30 @@ private enum Flag TIMESTAMP, NAMES_FOR_VALUES, KEYSPACE, - NOW_IN_SECONDS; + NOW_IN_SECONDS, + UNUSED_9, + UNUSED_10, + UNUSED_11, + UNUSED_12, + UNUSED_13, + UNUSED_14, + UNUSED_15, + UNUSED_16, + UNUSED_17, + UNUSED_18, + UNUSED_19, + UNUSED_20, + UNUSED_21, + UNUSED_22, + UNUSED_23, + UNUSED_24, + UNUSED_25, + UNUSED_26, + UNUSED_27, + UNUSED_28, + UNUSED_29, + PAGE_SIZE_IN_BYTES, + UNUSED_31; private static final Flag[] ALL_VALUES = values(); @@ -469,9 +493,13 @@ public QueryOptions decode(ByteBuf body, ProtocolVersion version) SpecificOptions options = SpecificOptions.DEFAULT; if (!flags.isEmpty()) { - int pageSize = flags.contains(Flag.PAGE_SIZE) ? body.readInt() : -1; + PageSize pageSize = flags.contains(Flag.PAGE_SIZE) + ? flags.contains(Flag.PAGE_SIZE_IN_BYTES) + ? PageSize.inBytes(body.readInt()) + : PageSize.inRows(body.readInt()) + : PageSize.NONE; PagingState pagingState = flags.contains(Flag.PAGING_STATE) ? PagingState.deserialize(CBUtil.readValueNoCopy(body), version) : null; - ConsistencyLevel serialConsistency = flags.contains(Flag.SERIAL_CONSISTENCY) ? CBUtil.readConsistencyLevel(body) : ConsistencyLevel.SERIAL; + ConsistencyLevel serialConsistency = flags.contains(Flag.SERIAL_CONSISTENCY) ? CBUtil.readConsistencyLevel(body) : null; long timestamp = Long.MIN_VALUE; if (flags.contains(Flag.TIMESTAMP)) { @@ -502,11 +530,11 @@ public void encode(QueryOptions options, ByteBuf dest, ProtocolVersion version) if (flags.contains(Flag.VALUES)) CBUtil.writeValueList(options.getValues(), dest); if (flags.contains(Flag.PAGE_SIZE)) - dest.writeInt(options.getPageSize()); + dest.writeInt(options.getPageSize().getSize()); if (flags.contains(Flag.PAGING_STATE)) CBUtil.writeValue(options.getPagingState().serialize(version), dest); if (flags.contains(Flag.SERIAL_CONSISTENCY)) - CBUtil.writeConsistencyLevel(options.getSerialConsistency(), dest); + CBUtil.writeConsistencyLevel(options.getSerialConsistency(null), dest); if (flags.contains(Flag.TIMESTAMP)) dest.writeLong(options.getSpecificOptions().timestamp); if (flags.contains(Flag.KEYSPACE)) @@ -535,7 +563,7 @@ public int encodedSize(QueryOptions options, ProtocolVersion version) if (flags.contains(Flag.PAGING_STATE)) size += CBUtil.sizeOfValue(options.getPagingState().serializedSize(version)); if (flags.contains(Flag.SERIAL_CONSISTENCY)) - size += CBUtil.sizeOfConsistencyLevel(options.getSerialConsistency()); + size += CBUtil.sizeOfConsistencyLevel(options.getSerialConsistency(null)); if (flags.contains(Flag.TIMESTAMP)) size += 8; if (flags.contains(Flag.KEYSPACE)) @@ -553,11 +581,11 @@ private EnumSet gatherFlags(QueryOptions options, ProtocolVersion version) flags.add(Flag.VALUES); if (options.skipMetadata()) flags.add(Flag.SKIP_METADATA); - if (options.getPageSize() >= 0) + if (options.getPageSize().isDefined()) flags.add(Flag.PAGE_SIZE); if (options.getPagingState() != null) flags.add(Flag.PAGING_STATE); - if (options.getSerialConsistency() != ConsistencyLevel.SERIAL) + if (options.getSpecificOptions().serialConsistency != null) flags.add(Flag.SERIAL_CONSISTENCY); if (options.getSpecificOptions().timestamp != Long.MIN_VALUE) flags.add(Flag.TIMESTAMP); @@ -568,6 +596,8 @@ private EnumSet gatherFlags(QueryOptions options, ProtocolVersion version) flags.add(Flag.KEYSPACE); if (options.getSpecificOptions().nowInSeconds != Integer.MIN_VALUE) flags.add(Flag.NOW_IN_SECONDS); + if (options.getSpecificOptions().pageSize.getUnit() == PageSize.PageUnit.BYTES) + flags.add(Flag.PAGE_SIZE_IN_BYTES); } return flags; diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java index 87829ab358ac..2383ee85a2bd 100644 --- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java +++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java @@ -207,7 +207,7 @@ public ResultMessage processStatement(CQLStatement statement, QueryState querySt logger.trace("Process {} @CL.{}", statement, options.getConsistency()); ClientState clientState = queryState.getClientState(); statement.authorize(clientState); - statement.validate(clientState); + statement.validate(queryState); ResultMessage result; if (options.getConsistency() == ConsistencyLevel.NODE_LOCAL) @@ -310,7 +310,7 @@ public static Prepared prepareInternal(String query) throws RequestValidationExc // Note: if 2 threads prepare the same query, we'll live so don't bother synchronizing CQLStatement statement = parseStatement(query, internalQueryState().getClientState()); - statement.validate(internalQueryState().getClientState()); + statement.validate(internalQueryState()); prepared = new Prepared(statement); internalStatements.put(query, prepared); @@ -351,14 +351,14 @@ public static UntypedResultSet execute(String query, ConsistencyLevel cl, QueryS } } - public static UntypedResultSet executeInternalWithPaging(String query, int pageSize, Object... values) + public static UntypedResultSet executeInternalWithPaging(String query, PageSize pageSize, Object... values) { Prepared prepared = prepareInternal(query); if (!(prepared.statement instanceof SelectStatement)) throw new IllegalArgumentException("Only SELECTs can be paged"); SelectStatement select = (SelectStatement)prepared.statement; - QueryPager pager = select.getQuery(makeInternalOptions(prepared.statement, values), FBUtilities.nowInSeconds()).getPager(null, ProtocolVersion.CURRENT); + QueryPager pager = select.getQuery(QueryState.forInternalCalls(), makeInternalOptions(prepared.statement, values), FBUtilities.nowInSeconds()).getPager(null, ProtocolVersion.CURRENT); return UntypedResultSet.create(select, pager, pageSize); } @@ -385,7 +385,7 @@ public static UntypedResultSet executeOnceInternalWithNowAndTimestamp(int nowInS private static UntypedResultSet executeOnceInternal(QueryState queryState, String query, Object... values) { CQLStatement statement = parseStatement(query, queryState.getClientState()); - statement.validate(queryState.getClientState()); + statement.validate(queryState); ResultMessage result = statement.executeLocally(queryState, makeInternalOptions(statement, values)); if (result instanceof ResultMessage.Rows) return UntypedResultSet.create(((ResultMessage.Rows)result).result); @@ -536,7 +536,7 @@ public ResultMessage processBatch(BatchStatement batch, QueryState queryState, B ClientState clientState = queryState.getClientState().cloneWithKeyspaceIfSet(options.getKeyspace()); batch.authorize(clientState); batch.validate(); - batch.validate(clientState); + batch.validate(queryState); return batch.execute(queryState, options, queryStartNanoTime); } diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java index aab4192587fb..f6b33e30902e 100644 --- a/src/java/org/apache/cassandra/cql3/Sets.java +++ b/src/java/org/apache/cassandra/cql3/Sets.java @@ -24,6 +24,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.db.DecoratedKey; @@ -119,8 +120,8 @@ public static String setToString(Iterable items, java.util.function.Funct public static AbstractType getExactSetTypeIfKnown(List items, java.util.function.Function> mapper) { - Optional> type = items.stream().map(mapper).filter(Objects::nonNull).findFirst(); - return type.isPresent() ? SetType.getInstance(type.get(), false) : null; + AbstractType type = Lists.getElementType(items, mapper); + return type != null ? SetType.getInstance(type, false) : null; } public static class Literal extends Term.Raw @@ -338,6 +339,7 @@ public Adder(ColumnMetadata column, Term t) super(column, t); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to add items to a frozen set"; @@ -348,26 +350,40 @@ public void execute(DecoratedKey partitionKey, UpdateParameters params) throws I static void doAdd(Term.Terminal value, ColumnMetadata column, UpdateParameters params) throws InvalidRequestException { + if (value == null) + { + // for frozen sets, we're overwriting the whole cell + if (!column.type.isMultiCell()) + params.addTombstone(column); + + return; + } + + Set elements = ((Value) value).elements; + if (column.type.isMultiCell()) { - if (value == null) - return; + // Guardrails about collection size are only checked for the added elements without considering + // already existent elements. This is done so to avoid read-before-write, having additional checks + // during SSTable write. + Guardrails.itemsPerCollection.guard(elements.size(), column.name.toString(), false, params.state); - for (ByteBuffer bb : ((Value) value).elements) + int dataSize = 0; + for (ByteBuffer bb : elements) { if (bb == ByteBufferUtil.UNSET_BYTE_BUFFER) continue; - params.addCell(column, CellPath.create(bb), ByteBufferUtil.EMPTY_BYTE_BUFFER); + Cell cell = params.addCell(column, CellPath.create(bb), ByteBufferUtil.EMPTY_BYTE_BUFFER); + dataSize += cell.dataSize(); } + Guardrails.collectionSize.guard(dataSize, column.name.toString(), false, params.state); } else { - // for frozen sets, we're overwriting the whole cell - if (value == null) - params.addTombstone(column); - else - params.addCell(column, value.get(ProtocolVersion.CURRENT)); + Guardrails.itemsPerCollection.guard(elements.size(), column.name.toString(), false, params.state); + Cell cell = params.addCell(column, value.get(ProtocolVersion.CURRENT)); + Guardrails.collectionSize.guard(cell.dataSize(), column.name.toString(), false, params.state); } } } @@ -380,6 +396,7 @@ public Discarder(ColumnMetadata column, Term t) super(column, t); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to remove items from a frozen set"; @@ -405,6 +422,7 @@ public ElementDiscarder(ColumnMetadata column, Term k) super(column, k); } + @Override public void execute(DecoratedKey partitionKey, UpdateParameters params) throws InvalidRequestException { assert column.type.isMultiCell() : "Attempted to delete a single element in a frozen set"; diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java index 9ff3f075db53..d4e1915709f8 100644 --- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java +++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java @@ -150,7 +150,7 @@ public String toCQLString() entityAsString = String.format("%s[%s]", entityAsString, mapKey); if (isIN()) - return String.format("%s IN %s", entityAsString, Tuples.tupleToString(inValues)); + return String.format("%s IN %s", entityAsString, inValues == null ? value : Tuples.tupleToString(inValues)); return String.format("%s %s %s", entityAsString, relationType, value); } @@ -274,16 +274,6 @@ private List toReceivers(ColumnMetadata columnDef { ColumnSpecification receiver = columnDef; - if (isIN()) - { - // We only allow IN on the row key and the clustering key so far, never on non-PK columns, and this even if - // there's an index - // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that - // slide. - checkFalse(!columnDef.isPrimaryKeyColumn() && !canHaveOnlyOneValue(), - "IN predicates on non-primary-key columns (%s) is not yet supported", columnDef.name); - } - checkFalse(isContainsKey() && !(receiver.type instanceof MapType), "Cannot use CONTAINS KEY on non-map column %s", receiver.name); checkFalse(isContains() && !(receiver.type.isCollection()), "Cannot use CONTAINS on non-collection column %s", receiver.name); diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java index b8acd5954af4..6e028c274d31 100644 --- a/src/java/org/apache/cassandra/cql3/Tuples.java +++ b/src/java/org/apache/cassandra/cql3/Tuples.java @@ -154,14 +154,14 @@ public Value(ByteBuffer[] elements) public static Value fromSerialized(ByteBuffer bytes, TupleType type) { - ByteBuffer[] values = type.split(bytes); + ByteBuffer[] values = type.split(ByteBufferAccessor.instance, bytes); if (values.length > type.size()) { throw new InvalidRequestException(String.format( "Tuple value contained too many fields (expected %s, got %s)", type.size(), values.length)); } - return new Value(type.split(bytes)); + return new Value(type.split(ByteBufferAccessor.instance, bytes)); } public ByteBuffer get(ProtocolVersion protocolVersion) @@ -272,7 +272,8 @@ public static InValue fromSerialized(ByteBuffer value, ListType type, QueryOptio // type.split(bytes) List> elements = new ArrayList<>(l.size()); for (Object element : l) - elements.add(Arrays.asList(tupleType.split(type.getElementsType().decompose(element)))); + elements.add(Arrays.asList(tupleType.split(ByteBufferAccessor.instance, + type.getElementsType().decompose(element)))); return new InValue(elements); } catch (MarshalException e) diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java index f4ac99fdc4a4..7e386f765d43 100644 --- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java +++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java @@ -31,7 +31,7 @@ import org.apache.cassandra.db.partitions.PartitionIterator; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.pager.QueryPager; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.AbstractIterator; @@ -50,7 +50,7 @@ public static UntypedResultSet create(List> results) return new FromResultList(results); } - public static UntypedResultSet create(SelectStatement select, QueryPager pager, int pageSize) + public static UntypedResultSet create(SelectStatement select, QueryPager pager, PageSize pageSize) { return new FromPager(select, pager, pageSize); } @@ -62,11 +62,11 @@ public static UntypedResultSet create(SelectStatement select, QueryPager pager, @VisibleForTesting public static UntypedResultSet create(SelectStatement select, ConsistencyLevel cl, - ClientState clientState, + QueryState queryState, QueryPager pager, - int pageSize) + PageSize pageSize) { - return new FromDistributedPager(select, cl, clientState, pager, pageSize); + return new FromDistributedPager(select, cl, queryState, pager, pageSize); } public boolean isEmpty() @@ -168,10 +168,10 @@ private static class FromPager extends UntypedResultSet { private final SelectStatement select; private final QueryPager pager; - private final int pageSize; + private final PageSize pageSize; private final List metadata; - private FromPager(SelectStatement select, QueryPager pager, int pageSize) + private FromPager(SelectStatement select, QueryPager pager, PageSize pageSize) { this.select = select; this.pager = pager; @@ -227,19 +227,20 @@ private static class FromDistributedPager extends UntypedResultSet { private final SelectStatement select; private final ConsistencyLevel cl; - private final ClientState clientState; + private final QueryState queryState; private final QueryPager pager; - private final int pageSize; + private final PageSize pageSize; private final List metadata; private FromDistributedPager(SelectStatement select, ConsistencyLevel cl, - ClientState clientState, - QueryPager pager, int pageSize) + QueryState queryState, + QueryPager pager, + PageSize pageSize) { this.select = select; this.cl = cl; - this.clientState = clientState; + this.queryState = queryState; this.pager = pager; this.pageSize = pageSize; this.metadata = select.getResultMetadata().requestNames(); @@ -269,7 +270,7 @@ protected Row computeNext() if (pager.isExhausted()) return endOfData(); - try (PartitionIterator iter = pager.fetchPage(pageSize, cl, clientState, System.nanoTime())) + try (PartitionIterator iter = pager.fetchPage(pageSize, cl, queryState, System.nanoTime())) { currentPage = select.process(iter, nowInSec).rows.iterator(); } diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java index 427230734987..9767977b42b6 100644 --- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java +++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.Map; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.*; @@ -27,6 +28,7 @@ import org.apache.cassandra.db.partitions.Partition; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.service.QueryState; /** * Groups the parameters of an update query, and make building updates easier. @@ -36,6 +38,7 @@ public class UpdateParameters public final TableMetadata metadata; public final RegularAndStaticColumns updatedColumns; public final QueryOptions options; + public final QueryState state; private final int nowInSec; private final long timestamp; @@ -54,6 +57,7 @@ public class UpdateParameters public UpdateParameters(TableMetadata metadata, RegularAndStaticColumns updatedColumns, + QueryState state, QueryOptions options, long timestamp, int nowInSec, @@ -64,6 +68,7 @@ public UpdateParameters(TableMetadata metadata, this.metadata = metadata; this.updatedColumns = updatedColumns; this.options = options; + this.state = state; this.nowInSec = nowInSec; this.timestamp = timestamp; @@ -138,20 +143,29 @@ public void addTombstone(ColumnMetadata column) throws InvalidRequestException public void addTombstone(ColumnMetadata column, CellPath path) throws InvalidRequestException { + if (path != null && column.type.isMultiCell()) + Guardrails.columnValueSize.guard(path.dataSize(), column.name.toString(), false, state); + builder.addCell(BufferCell.tombstone(column, timestamp, nowInSec, path)); } - public void addCell(ColumnMetadata column, ByteBuffer value) throws InvalidRequestException + public Cell addCell(ColumnMetadata column, ByteBuffer value) throws InvalidRequestException { - addCell(column, null, value); + return addCell(column, null, value); } - public void addCell(ColumnMetadata column, CellPath path, ByteBuffer value) throws InvalidRequestException + public Cell addCell(ColumnMetadata column, CellPath path, ByteBuffer value) throws InvalidRequestException { + Guardrails.columnValueSize.guard(value.remaining(), column.name.toString(), false, state); + + if (path != null && column.type.isMultiCell()) + Guardrails.columnValueSize.guard(path.dataSize(), column.name.toString(), false, state); + Cell cell = ttl == LivenessInfo.NO_TTL ? BufferCell.live(column, timestamp, value, path) : BufferCell.expiring(column, timestamp, ttl, nowInSec, value, path); builder.addCell(cell); + return cell; } public void addCounter(ColumnMetadata column, long increment) throws InvalidRequestException diff --git a/src/java/org/apache/cassandra/cql3/UserTypes.java b/src/java/org/apache/cassandra/cql3/UserTypes.java index b023a8a0b8f6..a63420fca3cd 100644 --- a/src/java/org/apache/cassandra/cql3/UserTypes.java +++ b/src/java/org/apache/cassandra/cql3/UserTypes.java @@ -217,7 +217,7 @@ public Value(UserType type, ByteBuffer[] elements) public static Value fromSerialized(ByteBuffer bytes, UserType type) { type.validate(bytes); - return new Value(type, type.split(bytes)); + return new Value(type, type.split(ByteBufferAccessor.instance, bytes)); } public ByteBuffer get(ProtocolVersion protocolVersion) diff --git a/src/java/org/apache/cassandra/cql3/WhereClause.java b/src/java/org/apache/cassandra/cql3/WhereClause.java index 16116a2b8ad2..1969283ec43c 100644 --- a/src/java/org/apache/cassandra/cql3/WhereClause.java +++ b/src/java/org/apache/cassandra/cql3/WhereClause.java @@ -17,30 +17,29 @@ */ package org.apache.cassandra.cql3; -import java.util.List; -import java.util.Objects; +import java.util.*; +import java.util.function.Predicate; +import java.util.stream.Collectors; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import org.antlr.runtime.RecognitionException; import org.apache.cassandra.cql3.restrictions.CustomIndexExpression; -import static java.lang.String.join; - -import static com.google.common.collect.Iterables.concat; -import static com.google.common.collect.Iterables.transform; - +/** + * This is a parsed representation of the expression following the WHERE element + * in a CQL statement. It is parsed into an arbitrary sized expression tree consisting + * of ExpressionElement elements. + */ public final class WhereClause { - private static final WhereClause EMPTY = new WhereClause(new Builder()); + private static final WhereClause EMPTY = new WhereClause(new AndElement(Collections.emptyList())); - public final List relations; - public final List expressions; + private final ExpressionElement rootElement; - private WhereClause(Builder builder) + private WhereClause(ExpressionElement rootElement) { - relations = builder.relations.build(); - expressions = builder.expressions.build(); + this.rootElement = rootElement; } public static WhereClause empty() @@ -50,26 +49,33 @@ public static WhereClause empty() public boolean containsCustomExpressions() { - return !expressions.isEmpty(); + return rootElement.containsCustomExpressions(); + } + + public ExpressionElement root() + { + return rootElement; } /** * Renames identifiers in all relations + * * @param from the old identifier - * @param to the new identifier + * @param to the new identifier * @return a new WhereClause with with "from" replaced by "to" in all relations */ public WhereClause renameIdentifier(ColumnIdentifier from, ColumnIdentifier to) { - WhereClause.Builder builder = new WhereClause.Builder(); - - relations.stream() - .map(r -> r.renameIdentifier(from, to)) - .forEach(builder::add); - - expressions.forEach(builder::add); + return new WhereClause(rootElement.rename(from, to)); + } - return builder.build(); + /** + * @return a new WhereClause with the expression tree transforemd into conjuntive form + * @see ExpressionElement#conjunctiveForm() + */ + public WhereClause conjunctiveForm() + { + return new WhereClause(rootElement.conjunctiveForm()); } public static WhereClause parse(String cql) throws RecognitionException @@ -90,9 +96,7 @@ public String toString() */ public String toCQLString() { - return join(" AND ", - concat(transform(relations, Relation::toCQLString), - transform(expressions, CustomIndexExpression::toCQLString))); + return rootElement.toString(); } @Override @@ -105,35 +109,497 @@ public boolean equals(Object o) return false; WhereClause wc = (WhereClause) o; - return relations.equals(wc.relations) && expressions.equals(wc.expressions); + return rootElement.toString().equals(wc.rootElement.toString()); } @Override public int hashCode() { - return Objects.hash(relations, expressions); + return Objects.hash(rootElement); } + /** + * This receives fragments from the parse operation and builds them into the final WhereClause. + * + * The received fragments are: + *
    + *
  • add(Relation) - adds a new relation to the current ParseState
  • + *
  • add(CustomIndexExpression) - adds a new custom index expression to the current ParseState
  • + *
  • startEnclosure - responds to a '(' and pushes the current ParseState onto the precedence stack
  • + *
  • endEnclosure - responds to a ')' and pulls the ParseState associated with the + * matching startEnclosure. It will pull any intermediate precedence states off the stack until it + * reaches the matching enclosure state
  • + *
  • setCurrentOperator - changes the operator in the ParseState. If this new operator is + * of a higher precedence than the current operator, the last expression is popped from the ParseState and + * the state is pushed onto the precedence stack
  • + *
  • build - always the last call. This builds the resultant ExpressionTree from the + * precedence stack and the current ParseState
  • + *
+ */ public static final class Builder { - ImmutableList.Builder relations = new ImmutableList.Builder<>(); - ImmutableList.Builder expressions = new ImmutableList.Builder<>(); + private final Deque precedenceStack = new ArrayDeque<>(); + private ParseState parseState = new ParseState(); + + public void add(Relation relation) + { + parseState.push(new RelationElement(relation)); + } + + public void add(CustomIndexExpression customIndexExpression) + { + parseState.push(new CustomIndexExpressionElement(customIndexExpression)); + } + + public void startEnclosure() + { + pushStack(PushState.ENCLOSURE); + } + + public void endEnclosure() + { + do + { + ExpressionElement expression = generate(); + parseState = precedenceStack.pop(); + parseState.push(expression); + } + while (parseState.enclosure == PushState.PRECEDENCE); + } + + public void setCurrentOperator(String value) + { + Operator operator = Operator.valueOf(value.toUpperCase()); + if (parseState.isChangeOfOperator(operator)) + { + if (parseState.higherPrecedence(operator)) + { + // Where we have a = 1 OR b = 1 AND c = 1. When the operator changes to AND + // we need to pop b = 1 from the parseState, push the parseState containing + // a = 1 OR and then add b = 1 to the new parseState + ExpressionElement last = parseState.pop(); + pushStack(PushState.PRECEDENCE); + parseState.push(last); + } + else + { + ExpressionElement element = generate(); + if (!precedenceStack.isEmpty() && precedenceStack.peek().enclosure == PushState.PRECEDENCE) + parseState = precedenceStack.pop(); + else + parseState.clear(); + parseState.push(element); + } + } + parseState.operator = operator; + } + + public WhereClause build() + { + while (!precedenceStack.isEmpty()) + { + ExpressionElement expression = generate(); + parseState = precedenceStack.pop(); + parseState.push(expression); + } + return new WhereClause(generate()); + } + + private void pushStack(PushState enclosure) + { + parseState.enclosure = enclosure; + precedenceStack.push(parseState); + parseState = new ParseState(); + } + + private ExpressionElement generate() + { + if (parseState.size() == 1) + return parseState.pop(); + return parseState.asContainer(); + } + } + + /** + * Represents the state of the parsing operation at a point of enclosure or precedence change. + */ + public static class ParseState + { + Operator operator = Operator.NONE; + PushState enclosure = PushState.NONE; + Deque expressionElements = new ArrayDeque<>(); + + void push(ExpressionElement element) + { + expressionElements.add(element); + } + + ExpressionElement pop() + { + return expressionElements.removeLast(); + } - public Builder add(Relation relation) + int size() { - relations.add(relation); + return expressionElements.size(); + } + + ParseState clear() + { + expressionElements.clear(); return this; } - public Builder add(CustomIndexExpression expression) + boolean isChangeOfOperator(Operator operator) + { + return this.operator != operator && expressionElements.size() > 1; + } + + boolean higherPrecedence(Operator operator) + { + return operator.compareTo(this.operator) > 0; + } + + ContainerElement asContainer() + { + return operator == Operator.OR + ? new OrElement(expressionElements) + : new AndElement(expressionElements); + } + } + + enum Operator + { + NONE, OR, AND; + + public String joinValue() + { + return " " + name() + " "; + } + } + + /** + * This is the reason why the ParseState was pushed onto the precedence stack. + */ + enum PushState + { + NONE, PRECEDENCE, ENCLOSURE + } + + public static abstract class ExpressionElement + { + public List operations() + { + return Collections.emptyList(); + } + + public boolean isDisjunction() + { + return false; + } + + public List relations() + { + return Collections.emptyList(); + } + + public List expressions() + { + return Collections.emptyList(); + } + + /** + * Returns true if the given function f evaluates to true on any of the expression tree nodes. + */ + public abstract boolean exists(Predicate f); + + /** + * Returns true if this expression tree contains more than one relation. + */ + public final boolean isCompound() + { + return exists(e -> e instanceof ContainerElement && ((ContainerElement) e).children.size() > 1); + } + + /** + * Returns true if this expression tree contains a CustomIndexExpressionElement node. + */ + public final boolean containsCustomExpressions() + { + return exists(CustomIndexExpressionElement.class::isInstance); + } + + public ExpressionElement rename(ColumnIdentifier from, ColumnIdentifier to) { - expressions.add(expression); return this; } - public WhereClause build() + /** + * Collapses expression tree levels of the same type to form a semantically equivalent, + * but simpler form of this tree. + * + * Collapsing is possible because OR and AND operations are associative. + * + *

+ * Examples: + *

+         * AND(a, AND(b, c))      -> AND(a, b, c)
+         * OR(OR(a, b), OR(c, d)) -> OR(a, b, c, d)
+         * AND(a, OR(b, c))       -> AND(a, OR(b, c))
+         * 
+ *

+ * + * @return a new tree; this tree is left unmodified + */ + public ExpressionElement flatten() + { + return this; + } + + /** + * Creates a new tree that is a conjunctive form of this tree, semantically equivalent to this tree. + * The root of the conjunctive form is always an AndElement. + * + * The result tree is flattened so that nested conjunctions are lifted up to become the direct + * children of the root element. If the original tree does not have a top-level AndElement, + * an AndElement is inserted at the top, and a flattened original tree becomes its only child. + * + *

+ * Examples: + *

+         * a = 1                                 -> AND(a = 1)
+         * AND()                                 -> AND()
+         * AND(a = 1, b = 2)                     -> AND(a = 1, b = 2)
+         * AND(a = 1, AND(b = 2, c = 3))         -> AND(a = 1, b = 2, c = 3)
+         * OR(a = 1, b = 2)                      -> AND(OR(a = 1, b = 2))
+         * OR(a = 1, OR(b = 2, c = 3))           -> AND(OR(a = 1, b = 2, c = 3))
+         * 
+ *

+ * + * @return a new tree; this tree is left unmodified + */ + public final AndElement conjunctiveForm() + { + ExpressionElement flattened = this.flatten(); + return flattened instanceof AndElement + ? (AndElement) flattened + : new AndElement(Lists.newArrayList(flattened)); + } + } + + public static abstract class VariableElement extends ExpressionElement + { + @Override + public boolean exists(Predicate f) + { + return f.test(this); + } + } + + public static class RelationElement extends VariableElement + { + private final Relation relation; + + public RelationElement(Relation relation) + { + this.relation = relation; + } + + @Override + public List relations() + { + return Lists.newArrayList(relation); + } + + @Override + public ExpressionElement rename(ColumnIdentifier from, ColumnIdentifier to) + { + return new RelationElement(relation.renameIdentifier(from, to)); + } + + @Override + public String toString() + { + return relation.toString(); + } + } + + public static class CustomIndexExpressionElement extends VariableElement + { + private final CustomIndexExpression customIndexExpression; + + public CustomIndexExpressionElement(CustomIndexExpression customIndexExpression) + { + this.customIndexExpression = customIndexExpression; + } + + @Override + public List expressions() + { + return Lists.newArrayList(customIndexExpression); + } + + @Override + public String toString() + { + return customIndexExpression.toString(); + } + } + + public static abstract class ContainerElement extends ExpressionElement + { + protected final List children; + + protected ContainerElement(Collection children) + { + this.children = new ArrayList<>(children.size()); + this.children.addAll(children); + } + + /** + * Returns a new container of the same type with new children copied from the given collection + */ + protected abstract ContainerElement withChildren(Collection children); + + protected abstract Operator operator(); + + protected abstract String emptyValue(); + + @Override + public List operations() + { + return children.stream() + .filter(c -> (c instanceof ContainerElement)) + .map(r -> ((ContainerElement) r)) + .collect(Collectors.toList()); + } + + @Override + public List relations() + { + return children.stream() + .filter(c -> (c instanceof RelationElement)) + .map(r -> (((RelationElement) r).relation)) + .collect(Collectors.toList()); + } + + @Override + public List expressions() + { + return children.stream() + .filter(c -> (c instanceof CustomIndexExpressionElement)) + .map(r -> (((CustomIndexExpressionElement) r).customIndexExpression)) + .collect(Collectors.toList()); + } + + @Override + public boolean exists(Predicate f) + { + return f.test(this) || children.stream().anyMatch(f); + } + + @Override + public ExpressionElement rename(ColumnIdentifier from, ColumnIdentifier to) + { + List newChildren = children + .stream() + .map(c -> c.rename(from, to)) + .collect(Collectors.toList()); + + return this.withChildren(newChildren); + } + + @Override + public ExpressionElement flatten() + { + List newChildren = new ArrayList<>(); + for (ExpressionElement child: children) + { + ExpressionElement flattened = child.flatten(); + newChildren.add(flattened); + + if (flattened instanceof ContainerElement) + { + ContainerElement ce = (ContainerElement) flattened; + if (ce.operator() == this.operator()) + { + newChildren.remove(newChildren.size() - 1); + newChildren.addAll(ce.children); + } + } + } + + return this.withChildren(newChildren); + } + + @Override + public String toString() + { + if (children.isEmpty()) + return emptyValue(); + + return children + .stream() + .map(c -> children.size() > 1 && c.isCompound() ? '(' + c.toString() + ')' : c.toString()) + .collect(Collectors.joining(operator().joinValue())); + } + } + + public static class AndElement extends ContainerElement + { + public AndElement(Collection children) + { + super(children); + } + + @Override + protected AndElement withChildren(Collection children) + { + return new AndElement(children); + } + + @Override + protected Operator operator() + { + return Operator.AND; + } + + @Override + protected String emptyValue() { - return new WhereClause(this); + return "TRUE"; + } + } + + public static class OrElement extends ContainerElement + { + public OrElement(Collection children) + { + super(children); + } + + @Override + protected OrElement withChildren(Collection children) + { + return new OrElement(children); + } + + @Override + protected Operator operator() + { + return Operator.OR; + } + + @Override + protected String emptyValue() + { + return "FALSE"; + } + + @Override + public boolean isDisjunction() + { + return true; } } } diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java index 93ed6ae941bd..8e34f6ecc2a3 100644 --- a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java +++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java @@ -612,10 +612,16 @@ private ByteBuffer rowValue(Row row) return cell == null ? null : cell.buffer(); } - Cell cell = getCell(row, column); + // getCell returns Cell, which requires a method call to properly convert. + return getCellBuffer(getCell(row, column), userType); + } + + private ByteBuffer getCellBuffer(Cell cell, UserType userType) + { return cell == null ? null - : userType.split(cell.buffer())[userType.fieldPosition(field)]; + : ByteBufferAccessor.instance.convert(userType.split(cell.accessor(), cell.value())[userType.fieldPosition(field)], + cell.accessor()); } private boolean isSatisfiedBy(ByteBuffer rowValue) diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java index 0a252ff557f0..af4e81465fd9 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java @@ -19,6 +19,7 @@ import java.util.*; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.QueryOptions; @@ -27,6 +28,7 @@ import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.index.IndexRegistry; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.utils.btree.BTreeSet; import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; @@ -42,71 +44,25 @@ final class ClusteringColumnRestrictions extends RestrictionSetWrapper */ protected final ClusteringComparator comparator; - /** - * true if filtering is allowed for this restriction, false otherwise - */ - private final boolean allowFiltering; - - public ClusteringColumnRestrictions(TableMetadata table) - { - this(table, false); - } - - public ClusteringColumnRestrictions(TableMetadata table, boolean allowFiltering) - { - this(table.comparator, new RestrictionSet(), allowFiltering); - } - private ClusteringColumnRestrictions(ClusteringComparator comparator, - RestrictionSet restrictionSet, - boolean allowFiltering) + RestrictionSet restrictionSet) { super(restrictionSet); this.comparator = comparator; - this.allowFiltering = allowFiltering; } - public ClusteringColumnRestrictions mergeWith(Restriction restriction) throws InvalidRequestException - { - SingleRestriction newRestriction = (SingleRestriction) restriction; - RestrictionSet newRestrictionSet = restrictions.addRestriction(newRestriction); - - if (!isEmpty() && !allowFiltering) - { - SingleRestriction lastRestriction = restrictions.lastRestriction(); - ColumnMetadata lastRestrictionStart = lastRestriction.getFirstColumn(); - ColumnMetadata newRestrictionStart = restriction.getFirstColumn(); - - checkFalse(lastRestriction.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position(), - "Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", - newRestrictionStart.name, - lastRestrictionStart.name); - - if (newRestrictionStart.position() < lastRestrictionStart.position() && newRestriction.isSlice()) - throw invalidRequest("PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", - restrictions.nextColumn(newRestrictionStart).name, - newRestrictionStart.name); - } - - return new ClusteringColumnRestrictions(this.comparator, newRestrictionSet, allowFiltering); - } - - private boolean hasMultiColumnSlice() - { - for (SingleRestriction restriction : restrictions) - { - if (restriction.isMultiColumn() && restriction.isSlice()) - return true; - } - return false; - } - - public NavigableSet> valuesAsClustering(QueryOptions options) throws InvalidRequestException + public NavigableSet> valuesAsClustering(QueryOptions options, QueryState queryState) throws InvalidRequestException { MultiCBuilder builder = MultiCBuilder.create(comparator, hasIN()); - for (SingleRestriction r : restrictions) + List restrictions = restrictions(); + for (int i = 0; i < restrictions.size(); i++) { + SingleRestriction r = restrictions.get(i); r.appendTo(builder, options); + + if (hasIN() && Guardrails.inSelectCartesianProduct.enabled(queryState)) + Guardrails.inSelectCartesianProduct.guard(builder.buildSize(), "IN Select", false, queryState); + if (builder.hasMissingElements()) break; } @@ -115,11 +71,14 @@ public NavigableSet> valuesAsClustering(QueryOptions options) thro public NavigableSet> boundsAsClustering(Bound bound, QueryOptions options) throws InvalidRequestException { - MultiCBuilder builder = MultiCBuilder.create(comparator, hasIN() || hasMultiColumnSlice()); + List restrictionsList = restrictions(); + + MultiCBuilder builder = MultiCBuilder.create(comparator, hasIN() || restrictions.hasMultiColumnSlice()); int keyPosition = 0; - for (SingleRestriction r : restrictions) + for (int i = 0; i < restrictionsList.size(); i++) { + SingleRestriction r = restrictionsList.get(i); if (handleInFilter(r, keyPosition)) break; @@ -144,50 +103,20 @@ public NavigableSet> boundsAsClustering(Bound bound, QueryOpt return builder.buildBound(bound.isStart(), true); } - /** - * Checks if any of the underlying restriction is a CONTAINS or CONTAINS KEY. - * - * @return true if any of the underlying restriction is a CONTAINS or CONTAINS KEY, - * false otherwise - */ - public final boolean hasContains() - { - for (SingleRestriction restriction : restrictions) - { - if (restriction.isContains()) - return true; - } - return false; - } - - /** - * Checks if any of the underlying restriction is a slice restrictions. - * - * @return true if any of the underlying restriction is a slice restrictions, - * false otherwise - */ - public final boolean hasSlice() - { - for (SingleRestriction restriction : restrictions) - { - if (restriction.isSlice()) - return true; - } - return false; - } - /** * Checks if underlying restrictions would require filtering * * @return true if any underlying restrictions require filtering, false * otherwise */ - public final boolean needFiltering() + public boolean needFiltering() { int position = 0; - for (SingleRestriction restriction : restrictions) + List restrictions = restrictions(); + for (int i = 0; i < restrictions.size(); i++) { + SingleRestriction restriction = restrictions.get(i); if (handleInFilter(restriction, position)) return true; @@ -198,18 +127,20 @@ public final boolean needFiltering() } @Override - public void addRowFilterTo(RowFilter filter, + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) throws InvalidRequestException { int position = 0; - for (SingleRestriction restriction : restrictions) + List restrictions = restrictions(); + for (int i = 0; i < restrictions.size(); i++) { + SingleRestriction restriction = restrictions.get(i); // We ignore all the clustering columns that can be handled by slices. if (handleInFilter(restriction, position) || restriction.hasSupportingIndex(indexRegistry)) { - restriction.addRowFilterTo(filter, indexRegistry, options); + restriction.addToRowFilter(filter, indexRegistry, options); continue; } @@ -223,4 +154,69 @@ private boolean handleInFilter(SingleRestriction restriction, int index) return restriction.isContains() || restriction.isLIKE() || index != restriction.getFirstColumn().position(); } + public static ClusteringColumnRestrictions.Builder builder(TableMetadata table, boolean allowFiltering) + { + return new Builder(table, allowFiltering, null); + } + + public static ClusteringColumnRestrictions.Builder builder(TableMetadata table, boolean allowFiltering, IndexRegistry indexRegistry) + { + return new Builder(table, allowFiltering, indexRegistry); + } + + public static class Builder + { + private final TableMetadata table; + private final boolean allowFiltering; + private final IndexRegistry indexRegistry; + + private final RestrictionSet.Builder restrictions = RestrictionSet.builder(); + + private Builder(TableMetadata table, boolean allowFiltering, IndexRegistry indexRegistry) + { + this.table = table; + this.allowFiltering = allowFiltering; + this.indexRegistry = indexRegistry; + } + + public ClusteringColumnRestrictions.Builder addRestriction(Restriction restriction) + { + return addRestriction(restriction, false); + } + + public ClusteringColumnRestrictions.Builder addRestriction(Restriction restriction, boolean isDisjunction) + { + SingleRestriction newRestriction = (SingleRestriction) restriction; + boolean isEmpty = restrictions.isEmpty(); + + if (!isEmpty && !allowFiltering && (indexRegistry == null || !newRestriction.hasSupportingIndex(indexRegistry))) + { + SingleRestriction lastRestriction = restrictions.lastRestriction(); + ColumnMetadata lastRestrictionStart = lastRestriction.getFirstColumn(); + ColumnMetadata newRestrictionStart = newRestriction.getFirstColumn(); + restrictions.addRestriction(newRestriction, isDisjunction); + + checkFalse(lastRestriction.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position(), + "Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", + newRestrictionStart.name, + lastRestrictionStart.name); + + if (newRestrictionStart.position() < lastRestrictionStart.position() && newRestriction.isSlice()) + throw invalidRequest("PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", + restrictions.nextColumn(newRestrictionStart).name, + newRestrictionStart.name); + } + else + { + restrictions.addRestriction(newRestriction, isDisjunction); + } + + return this; + } + + public ClusteringColumnRestrictions build() + { + return new ClusteringColumnRestrictions(table.comparator, restrictions.build()); + } + } } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java b/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java index 7a5fff62b4c4..7debf1ae3e5d 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java @@ -22,6 +22,7 @@ import org.apache.cassandra.cql3.*; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.Index; import org.apache.cassandra.schema.TableMetadata; public class CustomIndexExpression @@ -46,7 +47,7 @@ public void prepareValue(TableMetadata table, AbstractType expressionType, Va value.collectMarkerSpecification(boundNames); } - public void addToRowFilter(RowFilter filter, TableMetadata table, QueryOptions options) + public void addToRowFilter(RowFilter.Builder filter, TableMetadata table, QueryOptions options) { filter.addCustomIndexExpression(table, table.indexes @@ -60,6 +61,17 @@ public String toCQLString() return String.format("expr(%s,%s)", targetIndex.toCQLString(), valueRaw.getText()); } + public boolean needsFiltering(Index.Group indexGroup) + { + String indexName = targetIndex.getName(); + + for (Index index : indexGroup.getIndexes()) + if (index.getIndexMetadata().name.equals(indexName)) + return false; + + return true; + } + @Override public String toString() { diff --git a/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java index fd89d1b47000..812e1816bf90 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java @@ -19,48 +19,149 @@ package org.apache.cassandra.cql3.restrictions; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.apache.cassandra.cql3.QualifiedName; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.IndexRegistry; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; -public class IndexRestrictions +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; + +public final class IndexRestrictions { + /** + * The empty {@code IndexRestrictions}. + */ + private static final IndexRestrictions EMPTY_RESTRICTIONS = new IndexRestrictions(Collections.EMPTY_LIST, Collections.EMPTY_LIST); + public static final String INDEX_NOT_FOUND = "Invalid index expression, index %s not found for %s"; public static final String INVALID_INDEX = "Target index %s cannot be used to query %s"; public static final String CUSTOM_EXPRESSION_NOT_SUPPORTED = "Index %s does not support custom expressions"; public static final String NON_CUSTOM_INDEX_IN_EXPRESSION = "Only CUSTOM indexes may be used in custom index expressions, %s is not valid"; public static final String MULTIPLE_EXPRESSIONS = "Multiple custom index expressions in a single query are not supported"; - private final List regularRestrictions = new ArrayList<>(); - private final List customExpressions = new ArrayList<>(); + private final List regularRestrictions; + private final List externalRestrictions; + + private IndexRestrictions(List regularRestrictions, List externalExpressions) + { + this.regularRestrictions = regularRestrictions; + this.externalRestrictions = externalExpressions; + } - public void add(Restrictions restrictions) + /** + * Returns an empty {@code IndexRestrictions}. + * @return an empty {@code IndexRestrictions} + */ + public static IndexRestrictions of() { - regularRestrictions.add(restrictions); + return EMPTY_RESTRICTIONS; } - public void add(CustomIndexExpression expression) + /** + * Creates a new {@code IndexRestrictions.Builder} instance. + * @return a new {@code IndexRestrictions.Builder} instance. + */ + public static Builder builder() { - customExpressions.add(expression); + return new IndexRestrictions.Builder(); } public boolean isEmpty() { - return regularRestrictions.isEmpty() && customExpressions.isEmpty(); + return regularRestrictions.isEmpty() && externalRestrictions.isEmpty(); } + /** + * Returns the regular restrictions. + * @return the regular restrictions + */ public List getRestrictions() { return regularRestrictions; } - public List getCustomIndexExpressions() + /** + * Returns the external restrictions. + * @return the external restrictions + */ + public List getExternalExpressions() + { + return externalRestrictions; + } + + /** + * Returns the number of restrictions in external expression and regular restrictions. + * @return Returns the number of restrictions in external expression and regular restrictions. + */ + private int numOfSupportedRestrictions() + { + int numberOfRestrictions = getExternalExpressions().size(); + for (Restrictions restrictions : getRestrictions()) + numberOfRestrictions += restrictions.size(); + + return numberOfRestrictions; + } + + /** + * Returns whether these restrictions would need filtering if the specified index registry were used. + * + * @param indexRegistry an index registry + * @param hasClusteringColumnRestrictions {@code true} if there are restricted clustering columns + * @param hasMultipleContains {@code true} if there are multiple "contains" restrictions + * @return {@code true} if this would need filtering if {@code indexRegistry} were used, {@code false} otherwise + */ + public boolean needFiltering(IndexRegistry indexRegistry, boolean hasClusteringColumnRestrictions, boolean hasMultipleContains) + { + // We need filtering if any clustering columns have restrictions that are not supported + // by their indexes. + if (numOfSupportedRestrictions() == 0) + return hasClusteringColumnRestrictions; + + for (Index.Group group : indexRegistry.listIndexGroups()) + if (!needFiltering(group, hasMultipleContains)) + return false; + + return true; + } + + /** + * Returns whether these restrictions would need filtering if the specified index group were used. + * + * @param indexGroup an index group + * @param hasMultipleContains {@code true} if there are multiple "contains" restrictions + * @return {@code true} if this would need filtering if {@code indexGroup} were used, {@code false} otherwise + */ + private boolean needFiltering(Index.Group indexGroup, boolean hasMultipleContains) + { + if (hasMultipleContains && !indexGroup.supportsMultipleContains()) + return true; + + for (Restrictions restrictions : regularRestrictions) + if (restrictions.needsFiltering(indexGroup)) + return true; + + for (CustomIndexExpression restriction : externalRestrictions) + if (restriction.needsFiltering(indexGroup)) + return true; + + return false; + } + + public boolean indexBeingUsed(Index.Group indexGroup) { - return customExpressions; + for (Restrictions restrictions : regularRestrictions) + if (!restrictions.needsFiltering(indexGroup)) + return true; + + for (CustomIndexExpression restriction : externalRestrictions) + if (!restriction.needsFiltering(indexGroup)) + return true; + + return false; } static InvalidRequestException invalidIndex(QualifiedName indexName, TableMetadata table) @@ -75,17 +176,75 @@ static InvalidRequestException indexNotFound(QualifiedName indexName, TableMetad static InvalidRequestException nonCustomIndexInExpression(QualifiedName indexName) { - return new InvalidRequestException(String.format(NON_CUSTOM_INDEX_IN_EXPRESSION, indexName.getName())); + return invalidRequest(NON_CUSTOM_INDEX_IN_EXPRESSION, indexName.getName()); } static InvalidRequestException customExpressionNotSupported(QualifiedName indexName) { - return new InvalidRequestException(String.format(CUSTOM_EXPRESSION_NOT_SUPPORTED, indexName.getName())); + return invalidRequest(CUSTOM_EXPRESSION_NOT_SUPPORTED, indexName.getName()); } - - @Override - public String toString() + + /** + * Builder for IndexRestrictions. + */ + public static final class Builder { - return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE); + /** + * Builder for the regular restrictions. + */ + private List regularRestrictions = new ArrayList<>(); + + /** + * Builder for the custom expressions. + */ + private List externalRestrictions = new ArrayList<>(); + + private Builder() {} + + /** + * Adds the specified restrictions. + * + * @param restrictions the restrictions to add + * @return this {@code Builder} + */ + public Builder add(Restrictions restrictions) + { + regularRestrictions.add(restrictions); + return this; + } + + /** + * Adds the restrictions and custom expressions from the specified {@code IndexRestrictions}. + * + * @param restrictions the restrictions and custom expressions to add + * @return this {@code Builder} + */ + public Builder add(IndexRestrictions restrictions) + { + regularRestrictions.addAll(restrictions.regularRestrictions); + externalRestrictions.addAll(restrictions.externalRestrictions); + return this; + } + + /** + * Adds the specified index expression. + * + * @param restriction the index expression to add + * @return this {@code Builder} + */ + public Builder add(CustomIndexExpression restriction) + { + externalRestrictions.add(restriction); + return this; + } + + /** + * Builds a new {@code IndexRestrictions} instance + * @return a new {@code IndexRestrictions} instance + */ + public IndexRestrictions build() + { + return new IndexRestrictions(Collections.unmodifiableList(regularRestrictions), Collections.unmodifiableList(externalRestrictions)); + } } } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java index 4c6ce2f80edd..453b7e54693d 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java @@ -114,8 +114,27 @@ protected final String getColumnsInCommons(Restriction otherRestriction) public final boolean hasSupportingIndex(IndexRegistry indexRegistry) { for (Index index : indexRegistry.listIndexes()) - if (isSupportedBy(index)) - return true; + if (isSupportingIndex(index)) + return true; + return false; + } + + @Override + public boolean needsFiltering(Index.Group indexGroup) + { + for (ColumnMetadata column : columnDefs) + if (!isSupportedBy(indexGroup, column)) + return true; + + return false; + } + + private boolean isSupportedBy(Index.Group indexGroup, ColumnMetadata column) + { + for (Index index : indexGroup.getIndexes()) + if (isSupportedBy(index, column)) + return true; + return false; } @@ -126,7 +145,16 @@ public final boolean hasSupportingIndex(IndexRegistry indexRegistry) * @return true this type of restriction is supported by the specified index, * false otherwise. */ - protected abstract boolean isSupportedBy(Index index); + private boolean isSupportingIndex(Index index) + { + for (ColumnMetadata column : columnDefs) + if (isSupportedBy(index, column)) + return true; + + return false; + } + + protected abstract boolean isSupportedBy(Index index, ColumnMetadata def); public static class EQRestriction extends MultiColumnRestriction { @@ -164,12 +192,9 @@ public SingleRestriction doMergeWith(SingleRestriction otherRestriction) } @Override - protected boolean isSupportedBy(Index index) + protected boolean isSupportedBy(Index index, ColumnMetadata column) { - for(ColumnMetadata column : columnDefs) - if (index.supportsExpression(column, Operator.EQ)) - return true; - return false; + return index.supportsExpression(column, Operator.EQ); } @Override @@ -186,7 +211,7 @@ public MultiCBuilder appendTo(MultiCBuilder builder, QueryOptions options) } @Override - public final void addRowFilterTo(RowFilter filter, IndexRegistry indexRegistry, QueryOptions options) + public final void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { Tuples.Value t = ((Tuples.Value) value.bind(options)); List values = t.getElements(); @@ -234,16 +259,13 @@ public SingleRestriction doMergeWith(SingleRestriction otherRestriction) } @Override - protected boolean isSupportedBy(Index index) + protected boolean isSupportedBy(Index index, ColumnMetadata column) { - for (ColumnMetadata column: columnDefs) - if (index.supportsExpression(column, Operator.IN)) - return true; - return false; + return index.supportsExpression(column, Operator.IN); } @Override - public final void addRowFilterTo(RowFilter filter, + public final void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { @@ -416,12 +438,9 @@ public MultiCBuilder appendBoundTo(MultiCBuilder builder, Bound bound, QueryOpti } @Override - protected boolean isSupportedBy(Index index) + protected boolean isSupportedBy(Index index, ColumnMetadata column) { - for(ColumnMetadata def : columnDefs) - if (slice.isSupportedBy(def, index)) - return true; - return false; + return slice.isSupportedBy(column, index); } @Override @@ -472,7 +491,7 @@ public SingleRestriction doMergeWith(SingleRestriction otherRestriction) } @Override - public final void addRowFilterTo(RowFilter filter, + public final void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { @@ -546,12 +565,9 @@ public SingleRestriction doMergeWith(SingleRestriction otherRestriction) } @Override - protected boolean isSupportedBy(Index index) + protected boolean isSupportedBy(Index index, ColumnMetadata column) { - for(ColumnMetadata column : columnDefs) - if (index.supportsExpression(column, Operator.IS_NOT)) - return true; - return false; + return index.supportsExpression(column, Operator.IS_NOT); } @Override @@ -561,7 +577,7 @@ public MultiCBuilder appendTo(MultiCBuilder builder, QueryOptions options) } @Override - public final void addRowFilterTo(RowFilter filter, IndexRegistry indexRegistry, QueryOptions options) + public final void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { throw new UnsupportedOperationException("Secondary indexes do not support IS NOT NULL restrictions"); } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java index b1edf947fa75..85b038eacb28 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java @@ -23,6 +23,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.statements.Bound; +import org.apache.cassandra.service.QueryState; /** * A set of restrictions on the partition key. @@ -32,7 +33,7 @@ interface PartitionKeyRestrictions extends Restrictions { public PartitionKeyRestrictions mergeWith(Restriction restriction); - public List values(QueryOptions options); + public List values(QueryOptions options, QueryState queryState); public List bounds(Bound b, QueryOptions options); diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java index fbe5673c05ba..9e843ccce7f9 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java @@ -20,14 +20,16 @@ import java.nio.ByteBuffer; import java.util.*; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.statements.Bound; import org.apache.cassandra.db.ClusteringComparator; -import org.apache.cassandra.db.ClusteringPrefix; import org.apache.cassandra.db.MultiCBuilder; import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.index.IndexRegistry; +import org.apache.cassandra.service.QueryState; /** * A set of single restrictions on the partition key. @@ -42,27 +44,12 @@ final class PartitionKeySingleRestrictionSet extends RestrictionSetWrapper imple */ protected final ClusteringComparator comparator; - public PartitionKeySingleRestrictionSet(ClusteringComparator comparator) + private PartitionKeySingleRestrictionSet(RestrictionSet restrictionSet, ClusteringComparator comparator) { - super(new RestrictionSet()); + super(restrictionSet); this.comparator = comparator; } - private PartitionKeySingleRestrictionSet(PartitionKeySingleRestrictionSet restrictionSet, - SingleRestriction restriction) - { - super(restrictionSet.restrictions.addRestriction(restriction)); - this.comparator = restrictionSet.comparator; - } - - private List toByteBuffers(SortedSet clusterings) - { - List l = new ArrayList<>(clusterings.size()); - for (ClusteringPrefix clustering : clusterings) - l.add(clustering.serializeAsPartitionKey()); - return l; - } - @Override public PartitionKeyRestrictions mergeWith(Restriction restriction) { @@ -71,36 +58,52 @@ public PartitionKeyRestrictions mergeWith(Restriction restriction) if (isEmpty()) return (PartitionKeyRestrictions) restriction; - return new TokenFilter(this, (TokenRestriction) restriction); + return TokenFilter.create(this, (TokenRestriction) restriction); } - return new PartitionKeySingleRestrictionSet(this, (SingleRestriction) restriction); + Builder builder = PartitionKeySingleRestrictionSet.builder(comparator); + List restrictions = restrictions(); + for (int i = 0; i < restrictions.size(); i++) + { + SingleRestriction r = restrictions.get(i); + builder.addRestriction(r); + } + return builder.addRestriction(restriction) + .build(); } @Override - public List values(QueryOptions options) + public List values(QueryOptions options, QueryState queryState) { MultiCBuilder builder = MultiCBuilder.create(comparator, hasIN()); - for (SingleRestriction r : restrictions) + List restrictions = restrictions(); + for (int i = 0; i < restrictions.size(); i++) { + SingleRestriction r = restrictions.get(i); r.appendTo(builder, options); + + if (hasIN() && Guardrails.inSelectCartesianProduct.enabled(queryState)) + Guardrails.inSelectCartesianProduct.guard(builder.buildSize(), "IN Select", false, queryState); + if (builder.hasMissingElements()) break; } - return toByteBuffers(builder.build()); + return builder.buildSerializedPartitionKeys(); } @Override public List bounds(Bound bound, QueryOptions options) { MultiCBuilder builder = MultiCBuilder.create(comparator, hasIN()); - for (SingleRestriction r : restrictions) + List restrictions = restrictions(); + for (int i = 0; i < restrictions.size(); i++) { + SingleRestriction r = restrictions.get(i); r.appendBoundTo(builder, bound, options); if (builder.hasMissingElements()) - return Collections.emptyList(); + return Collections.EMPTY_LIST; } - return toByteBuffers(builder.buildBound(bound.isStart(), true)); + return builder.buildSerializedPartitionKeys(); } @Override @@ -120,13 +123,15 @@ public boolean isInclusive(Bound b) } @Override - public void addRowFilterTo(RowFilter filter, + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { - for (SingleRestriction restriction : restrictions) + List restrictions = restrictions(); + for (int i = 0; i < restrictions.size(); i++) { - restriction.addRowFilterTo(filter, indexRegistry, options); + SingleRestriction r = restrictions.get(i); + r.addToRowFilter(filter, indexRegistry, options); } } @@ -146,9 +151,65 @@ public boolean hasUnrestrictedPartitionKeyComponents(TableMetadata table) return size() < table.partitionKeyColumns().size(); } - @Override - public boolean hasSlice() + public static Builder builder(ClusteringComparator clusteringComparator) { - return restrictions.hasSlice(); + return new Builder(clusteringComparator); + } + + public static final class Builder + { + private final ClusteringComparator clusteringComparator; + + private final List restrictions = new ArrayList<>(); + + private Builder(ClusteringComparator clusteringComparator) { + this.clusteringComparator = clusteringComparator; + } + + public Builder addRestriction(Restriction restriction) + { + restrictions.add(restriction); + return this; + } + + public PartitionKeyRestrictions build() + { + return build(false); + } + + public PartitionKeyRestrictions build(boolean isDisjunction) + { + RestrictionSet.Builder restrictionSet = RestrictionSet.builder(); + + for (int i = 0; i < restrictions.size(); i++) { + Restriction restriction = restrictions.get(i); + + // restrictions on tokens are handled in a special way + if (restriction.isOnToken()) + return buildWithTokens(restrictionSet, i); + + restrictionSet.addRestriction((SingleRestriction) restriction, isDisjunction); + } + + return buildPartitionKeyRestrictions(restrictionSet); + } + + private PartitionKeyRestrictions buildWithTokens(RestrictionSet.Builder restrictionSet, int i) + { + PartitionKeyRestrictions merged = buildPartitionKeyRestrictions(restrictionSet); + + for (; i < restrictions.size(); i++) { + Restriction restriction = restrictions.get(i); + + merged = merged.mergeWith(restriction); + } + + return merged; + } + + private PartitionKeySingleRestrictionSet buildPartitionKeyRestrictions(RestrictionSet.Builder restrictionSet) + { + return new PartitionKeySingleRestrictionSet(restrictionSet.build(), clusteringComparator); + } } } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java index 91dedad9536b..bf181c085646 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java @@ -19,6 +19,7 @@ import java.util.List; +import org.apache.cassandra.index.Index; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.functions.Function; @@ -30,6 +31,10 @@ */ public interface Restriction { + /** + * Check if the restriction is on a partition key + * @return true if the restriction is on a partition key, false + */ public default boolean isOnToken() { return false; @@ -68,6 +73,14 @@ public default boolean isOnToken() */ public boolean hasSupportingIndex(IndexRegistry indexRegistry); + /** + * Returns whether this restriction would need filtering if the specified index group were used. + * + * @param indexGroup an index group + * @return {@code true} if this would need filtering if {@code indexGroup} were used, {@code false} otherwise + */ + public boolean needsFiltering(Index.Group indexGroup); + /** * Adds to the specified row filter the expressions corresponding to this Restriction. * @@ -75,7 +88,7 @@ public default boolean isOnToken() * @param indexRegistry the index registry * @param options the query options */ - public void addRowFilterTo(RowFilter filter, + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options); } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java index 7a5d5b964b0a..8f9f1e604126 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java @@ -18,259 +18,355 @@ package org.apache.cassandra.cql3.restrictions; import java.util.*; +import java.util.stream.Collectors; -import com.google.common.collect.AbstractIterator; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; -import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction.ContainsRestriction; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.Index; import org.apache.cassandra.index.IndexRegistry; -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; +import org.apache.cassandra.schema.ColumnMetadata; /** * Sets of column restrictions. * *

This class is immutable.

*/ -final class RestrictionSet implements Restrictions, Iterable +public abstract class RestrictionSet implements Restrictions { /** * The comparator used to sort the Restrictions. */ - private static final Comparator COLUMN_DEFINITION_COMPARATOR = new Comparator() + private static final Comparator COLUMN_DEFINITION_COMPARATOR = Comparator.comparingInt(ColumnMetadata::position).thenComparing(column -> column.name.bytes); + + private static final class EmptyRestrictionSet extends RestrictionSet { + private static final EmptyRestrictionSet INSTANCE = new EmptyRestrictionSet(); + + private EmptyRestrictionSet() + { + } + @Override - public int compare(ColumnMetadata column, ColumnMetadata otherColumn) + public void addToRowFilter(RowFilter.Builder rowFilter, IndexRegistry indexRegistry, QueryOptions options) throws InvalidRequestException { - int value = Integer.compare(column.position(), otherColumn.position()); - return value != 0 ? value : column.name.bytes.compareTo(otherColumn.name.bytes); } - }; - private static final TreeMap EMPTY = new TreeMap<>(COLUMN_DEFINITION_COMPARATOR); + @Override + public List getColumnDefs() + { + return Collections.EMPTY_LIST; + } - /** - * The restrictions per column. - */ - protected final TreeMap restrictions; + @Override + public void addFunctionsTo(List functions) + { + } - /** - * {@code true} if it contains multi-column restrictions, {@code false} otherwise. - */ - private final boolean hasMultiColumnRestrictions; + @Override + public boolean isEmpty() + { + return true; + } - private final boolean hasIn; - private final boolean hasContains; - private final boolean hasSlice; - private final boolean hasOnlyEqualityRestrictions; + @Override + public int size() + { + return 0; + } - public RestrictionSet() - { - this(EMPTY, false, - false, - false, - false, - true); - } + @Override + public boolean hasRestrictionFor(ColumnMetadata.Kind kind) + { + return false; + } - private RestrictionSet(TreeMap restrictions, - boolean hasMultiColumnRestrictions, - boolean hasIn, - boolean hasContains, - boolean hasSlice, - boolean hasOnlyEqualityRestrictions) - { - this.restrictions = restrictions; - this.hasMultiColumnRestrictions = hasMultiColumnRestrictions; - this.hasIn = hasIn; - this.hasContains = hasContains; - this.hasSlice = hasSlice; - this.hasOnlyEqualityRestrictions = hasOnlyEqualityRestrictions; - } + @Override + public Set getRestrictions(ColumnMetadata columnDef) + { + return Collections.emptySet(); + } - @Override - public void addRowFilterTo(RowFilter filter, IndexRegistry indexRegistry, QueryOptions options) throws InvalidRequestException - { - for (Restriction restriction : restrictions.values()) - restriction.addRowFilterTo(filter, indexRegistry, options); - } + @Override + public boolean hasSupportingIndex(IndexRegistry indexRegistry) + { + return false; + } - @Override - public List getColumnDefs() - { - return new ArrayList<>(restrictions.keySet()); - } + @Override + public boolean needsFiltering(Index.Group indexGroup) + { + return false; + } - @Override - public void addFunctionsTo(List functions) - { - for (Restriction restriction : this) - restriction.addFunctionsTo(functions); - } + @Override + public ColumnMetadata getFirstColumn() + { + return null; + } - @Override - public boolean isEmpty() - { - return restrictions.isEmpty(); - } + @Override + public ColumnMetadata getLastColumn() + { + return null; + } - @Override - public int size() - { - return restrictions.size(); - } + @Override + public SingleRestriction lastRestriction() + { + return null; + } - /** - * Checks if one of the restrictions applies to a column of the specific kind. - * @param kind the column kind - * @return {@code true} if one of the restrictions applies to a column of the specific kind, {@code false} otherwise. - */ - public boolean hasRestrictionFor(ColumnMetadata.Kind kind) - { - for (ColumnMetadata column : restrictions.keySet()) + @Override + public boolean hasMultipleContains() { - if (column.kind == kind) - return true; + return false; } - return false; - } - /** - * Adds the specified restriction to this set of restrictions. - * - * @param restriction the restriction to add - * @return the new set of restrictions - */ - public RestrictionSet addRestriction(SingleRestriction restriction) - { - // RestrictionSet is immutable so we need to clone the restrictions map. - TreeMap newRestricitons = new TreeMap<>(this.restrictions); - - boolean newHasIn = hasIn || restriction.isIN(); - boolean newHasContains = hasContains || restriction.isContains(); - boolean newHasSlice = hasSlice || restriction.isSlice(); - boolean newHasOnlyEqualityRestrictions = hasOnlyEqualityRestrictions && (restriction.isEQ() || restriction.isIN()); - - return new RestrictionSet(mergeRestrictions(newRestricitons, restriction), - hasMultiColumnRestrictions || restriction.isMultiColumn(), - newHasIn, - newHasContains, - newHasSlice, - newHasOnlyEqualityRestrictions); + @Override + public List restrictions() + { + return Collections.EMPTY_LIST; + } + + @Override + public boolean hasMultiColumnSlice() + { + return false; + } } - private TreeMap mergeRestrictions(TreeMap restrictions, - SingleRestriction restriction) + private static final class DefaultRestrictionSet extends RestrictionSet { - Collection columnDefs = restriction.getColumnDefs(); - Set existingRestrictions = getRestrictions(columnDefs); - if (existingRestrictions.isEmpty()) - { - for (ColumnMetadata columnDef : columnDefs) - restrictions.put(columnDef, restriction); - } - else + /** + * The keys from the 'restrictions' parameter to the + */ + private final List restrictionsKeys; + /** + * The values as returned from {@link #restrictions()}. + */ + private final List restrictionsValues; + private final Multimap restrictionsMap; + private final int hasBitmap; + private final int restrictionForKindBitmap; + private static final int maskHasContains = 1; + private static final int maskHasSlice = 2; + private static final int maskHasIN = 4; + private static final int maskHasOnlyEqualityRestrictions = 8; + private static final int maskHasMultiColumnSlice = 16; + private static final int maskHasMultipleContains = 32; + + private DefaultRestrictionSet(Multimap restrictions, + boolean hasMultiColumnRestrictions) { - for (SingleRestriction existing : existingRestrictions) + this.restrictionsKeys = new ArrayList<>(restrictions.keySet()); + restrictionsKeys.sort(COLUMN_DEFINITION_COMPARATOR); + + List sortedRestrictions = new ArrayList<>(); + + int numberOfContains = 0; + int restrictionForBitmap = 0; + int bitmap = maskHasOnlyEqualityRestrictions; + + SingleRestriction previous = null; + for (int i = 0; i < restrictionsKeys.size(); i++) { - SingleRestriction newRestriction = mergeRestrictions(existing, restriction); + ColumnMetadata col = restrictionsKeys.get(i); + Collection columnRestrictions = restrictions.get(col); - for (ColumnMetadata columnDef : columnDefs) - restrictions.put(columnDef, newRestriction); + for (SingleRestriction singleRestriction : columnRestrictions) + { + if (singleRestriction.isContains()) + { + bitmap |= maskHasContains; + ContainsRestriction contains = (ContainsRestriction) singleRestriction; + numberOfContains += (contains.numberOfValues() + contains.numberOfKeys() + contains.numberOfEntries()); + } + + if (hasMultiColumnRestrictions) + { + if (singleRestriction.equals(previous)) + continue; + previous = singleRestriction; + } + + restrictionForBitmap |= 1 << col.kind.ordinal(); + + sortedRestrictions.add(singleRestriction); + + if (singleRestriction.isSlice()) + { + bitmap |= maskHasSlice; + if (singleRestriction.isMultiColumn()) + bitmap |= maskHasMultiColumnSlice; + } + + if (singleRestriction.isIN()) + bitmap |= maskHasIN; + else if (!singleRestriction.isEQ()) + bitmap &= ~maskHasOnlyEqualityRestrictions; + } } + this.hasBitmap = bitmap | (numberOfContains > 1 ? maskHasMultipleContains : 0); + this.restrictionForKindBitmap = restrictionForBitmap; + + this.restrictionsValues = Collections.unmodifiableList(sortedRestrictions); + this.restrictionsMap = restrictions; } - return restrictions; - } + @Override + public void addToRowFilter(RowFilter.Builder rowFilter, + IndexRegistry indexRegistry, + QueryOptions options) throws InvalidRequestException + { + for (SingleRestriction restriction : restrictionsMap.values()) + restriction.addToRowFilter(rowFilter, indexRegistry, options); + } - @Override - public Set getRestrictions(ColumnMetadata columnDef) - { - Restriction existing = restrictions.get(columnDef); - return existing == null ? Collections.emptySet() : Collections.singleton(existing); - } + @Override + public List getColumnDefs() + { + return restrictionsKeys; + } - /** - * Returns all the restrictions applied to the specified columns. - * - * @param columnDefs the column definitions - * @return all the restrictions applied to the specified columns - */ - private Set getRestrictions(Collection columnDefs) - { - Set set = new HashSet<>(); - for (ColumnMetadata columnDef : columnDefs) + @Override + public void addFunctionsTo(List functions) { - SingleRestriction existing = restrictions.get(columnDef); - if (existing != null) - set.add(existing); + for (int i = 0; i < restrictionsValues.size(); i++) + restrictionsValues.get(i).addFunctionsTo(functions); } - return set; - } - @Override - public final boolean hasSupportingIndex(IndexRegistry indexRegistry) - { - for (Restriction restriction : restrictions.values()) + @Override + public boolean isEmpty() { - if (restriction.hasSupportingIndex(indexRegistry)) - return true; + return false; } - return false; - } - /** - * Returns the column after the specified one. - * - * @param columnDef the column for which the next one need to be found - * @return the column after the specified one. - */ - ColumnMetadata nextColumn(ColumnMetadata columnDef) - { - return restrictions.tailMap(columnDef, false).firstKey(); - } + @Override + public int size() + { + return restrictionsKeys.size(); + } - @Override - public ColumnMetadata getFirstColumn() - { - return isEmpty() ? null : this.restrictions.firstKey(); - } + @Override + public boolean hasRestrictionFor(ColumnMetadata.Kind kind) + { + return 0 != (restrictionForKindBitmap & 1 << kind.ordinal()); + } - @Override - public ColumnMetadata getLastColumn() - { - return isEmpty() ? null : this.restrictions.lastKey(); + @Override + public Set getRestrictions(ColumnMetadata columnDef) + { + return restrictionsMap.get(columnDef).stream().map(r -> ((Restriction)r)).collect(Collectors.toSet()); + } + + @Override + public boolean hasSupportingIndex(IndexRegistry indexRegistry) + { + for (SingleRestriction restriction : restrictionsMap.values()) + if (restriction.hasSupportingIndex(indexRegistry)) + return true; + return false; + } + + @Override + public boolean needsFiltering(Index.Group indexGroup) + { + for (SingleRestriction restriction : restrictionsMap.values()) + if (restriction.needsFiltering(indexGroup)) + return true; + + return false; + } + + @Override + public ColumnMetadata getFirstColumn() + { + return this.restrictionsKeys.get(0); + } + + @Override + public ColumnMetadata getLastColumn() + { + return this.restrictionsKeys.get(this.restrictionsKeys.size() - 1); + } + + @Override + public SingleRestriction lastRestriction() + { + return this.restrictionsValues.get(this.restrictionsValues.size() - 1); + } + + @Override + public boolean hasMultipleContains() + { + return 0 != (hasBitmap & maskHasMultipleContains); + } + + @Override + public List restrictions() + { + return restrictionsValues; + } + + @Override + public boolean hasIN() + { + return 0 != (hasBitmap & maskHasIN); + } + + @Override + public boolean hasContains() + { + return 0 != (hasBitmap & maskHasContains); + } + + @Override + public boolean hasSlice() + { + return 0 != (hasBitmap & maskHasSlice); + } + + @Override + public boolean hasMultiColumnSlice() + { + return 0 != (hasBitmap & maskHasMultiColumnSlice); + } + + @Override + public boolean hasOnlyEqualityRestrictions() + { + return 0 != (hasBitmap & maskHasOnlyEqualityRestrictions); + } } + /** + * Checks if one of the restrictions applies to a column of the specific kind. + * @param kind the column kind + * @return {@code true} if one of the restrictions applies to a column of the specific kind, {@code false} otherwise. + */ + public abstract boolean hasRestrictionFor(ColumnMetadata.Kind kind); + /** * Returns the last restriction. - * - * @return the last restriction. */ - SingleRestriction lastRestriction() - { - return isEmpty() ? null : this.restrictions.lastEntry().getValue(); - } + public abstract SingleRestriction lastRestriction(); /** - * Merges the two specified restrictions. + * Checks if the restrictions contains multiple contains, contains key, or map[key] = value. * - * @param restriction the first restriction - * @param otherRestriction the second restriction - * @return the merged restriction - * @throws InvalidRequestException if the two restrictions cannot be merged + * @return true if the restrictions contain multiple contains, contains key, or , + * map[key] = value; false otherwise */ - private static SingleRestriction mergeRestrictions(SingleRestriction restriction, - SingleRestriction otherRestriction) - { - return restriction == null ? otherRestriction - : restriction.mergeWith(otherRestriction); - } + public abstract boolean hasMultipleContains(); + + public abstract List restrictions(); /** * Checks if the restrictions contains multiple contains, contains key, or map[key] = value. @@ -278,98 +374,113 @@ private static SingleRestriction mergeRestrictions(SingleRestriction restriction * @return true if the restrictions contains multiple contains, contains key, or , * map[key] = value; false otherwise */ - public final boolean hasMultipleContains() - { - int numberOfContains = 0; - for (SingleRestriction restriction : restrictions.values()) - { - if (restriction.isContains()) - { - ContainsRestriction contains = (ContainsRestriction) restriction; - numberOfContains += (contains.numberOfValues() + contains.numberOfKeys() + contains.numberOfEntries()); - } - } - return numberOfContains > 1; - } + public abstract boolean hasMultiColumnSlice(); - @Override - public Iterator iterator() + public static Builder builder() { - Iterator iterator = restrictions.values().iterator(); - return hasMultiColumnRestrictions ? new DistinctIterator<>(iterator) : iterator; + return new Builder(); } - /** - * Checks if any of the underlying restriction is an IN. - * @return true if any of the underlying restriction is an IN, false otherwise - */ - public final boolean hasIN() + public static final class Builder { - return hasIn; - } + private final Multimap newRestrictions = ArrayListMultimap.create(); + private boolean multiColumn = false; - public boolean hasContains() - { - return hasContains; - } + private ColumnMetadata lastRestrictionColumn; + private SingleRestriction lastRestriction; - public final boolean hasSlice() - { - return hasSlice; - } + private Builder() + { + } - /** - * Checks if all of the underlying restrictions are EQ or IN restrictions. - * - * @return true if all of the underlying restrictions are EQ or IN restrictions, - * false otherwise - */ - public final boolean hasOnlyEqualityRestrictions() - { - return hasOnlyEqualityRestrictions; - } + public void addRestriction(SingleRestriction restriction, boolean isDisjunction) + { + List columnDefs = restriction.getColumnDefs(); - /** - * {@code Iterator} decorator that removes duplicates in an ordered one. - * - * @param iterator the decorated iterator - * @param the iterator element type. - */ - private static final class DistinctIterator extends AbstractIterator - { - /** - * The decorated iterator. - */ - private final Iterator iterator; + if (isDisjunction) + { + // If this restriction is part of a disjunction query then we don't want + // to merge the restrictions (if that is possible), we just add the + // restriction to the set of restrictions for the column. + addRestrictionForColumns(columnDefs, restriction, false); + } + else + { + // If this restriction isn't part of a disjunction then we need to get + // the set of existing restrictions for the column and merge them with the + // new restriction + Set existingRestrictions = getRestrictions(newRestrictions, columnDefs); - /** - * The previous element. - */ - private E previous; + if (existingRestrictions.isEmpty()) + { + addRestrictionForColumns(columnDefs, restriction, false); + } + else + { + for (SingleRestriction existing : existingRestrictions) + { + SingleRestriction newRestriction = existing.mergeWith(restriction); - public DistinctIterator(Iterator iterator) - { - this.iterator = iterator; + addRestrictionForColumns(columnDefs, newRestriction, true); + } + } + } } - protected E computeNext() + private void addRestrictionForColumns(List columnDefs, SingleRestriction restriction, boolean replace) { - while(iterator.hasNext()) + for (int i = 0; i < columnDefs.size(); i++) { - E next = iterator.next(); - if (!next.equals(previous)) + ColumnMetadata column = columnDefs.get(i); + if (lastRestrictionColumn == null || COLUMN_DEFINITION_COMPARATOR.compare(lastRestrictionColumn, column) < 0) { - previous = next; - return next; + lastRestrictionColumn = column; + lastRestriction = restriction; } + // If the restriction is a merger of new restriction and existing restrictions then + // we need to remove the existing restrictions for the column before adding it + if (replace) + newRestrictions.removeAll(column); + newRestrictions.put(column, restriction); } - return endOfData(); + + multiColumn |= restriction.isMultiColumn(); + } + + private static Set getRestrictions(Multimap restrictions, + List columnDefs) + { + Set set = new HashSet<>(); + for (int i = 0; i < columnDefs.size(); i++) + { + Collection existing = restrictions.get(columnDefs.get(i)); + if (!existing.isEmpty()) + set.addAll(existing); + } + return set; + } + + public RestrictionSet build() + { + return isEmpty() ? EmptyRestrictionSet.INSTANCE : new DefaultRestrictionSet(newRestrictions, multiColumn); + } + + public boolean isEmpty() + { + return newRestrictions.isEmpty(); + } + + public SingleRestriction lastRestriction() + { + return lastRestriction; + } + + public ColumnMetadata nextColumn(ColumnMetadata columnDef) + { + // This method is only invoked in the statement-preparation-phase to construct an error message. + NavigableSet columns = new TreeSet<>(COLUMN_DEFINITION_COMPARATOR); + columns.addAll(newRestrictions.keySet()); + return columns.tailSet(columnDef, false).first(); } - } - - @Override - public String toString() - { - return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE); } } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java index 9803adc4592f..b6c943ab065a 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java @@ -20,9 +20,7 @@ import java.util.List; import java.util.Set; -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; - +import org.apache.cassandra.index.Index; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.functions.Function; @@ -40,23 +38,26 @@ class RestrictionSetWrapper implements Restrictions */ protected final RestrictionSet restrictions; - public RestrictionSetWrapper(RestrictionSet restrictions) + RestrictionSetWrapper(RestrictionSet restrictions) { this.restrictions = restrictions; } - public void addRowFilterTo(RowFilter filter, + @Override + public void addToRowFilter(RowFilter.Builder rowFilter, IndexRegistry indexRegistry, QueryOptions options) { - restrictions.addRowFilterTo(filter, indexRegistry, options); + restrictions.addToRowFilter(rowFilter, indexRegistry, options); } + @Override public List getColumnDefs() { return restrictions.getColumnDefs(); } + @Override public void addFunctionsTo(List functions) { restrictions.addFunctionsTo(functions); @@ -67,54 +68,67 @@ public boolean isEmpty() return restrictions.isEmpty(); } + public List restrictions() + { + return restrictions.restrictions(); + } + public int size() { return restrictions.size(); } + @Override public boolean hasSupportingIndex(IndexRegistry indexRegistry) { return restrictions.hasSupportingIndex(indexRegistry); } + @Override + public boolean needsFiltering(Index.Group indexGroup) + { + return restrictions.needsFiltering(indexGroup); + } + + @Override public ColumnMetadata getFirstColumn() { return restrictions.getFirstColumn(); } + @Override public ColumnMetadata getLastColumn() { return restrictions.getLastColumn(); } + @Override public boolean hasIN() { return restrictions.hasIN(); } + @Override public boolean hasContains() { return restrictions.hasContains(); } + @Override public boolean hasSlice() { return restrictions.hasSlice(); } + @Override public boolean hasOnlyEqualityRestrictions() { return restrictions.hasOnlyEqualityRestrictions(); } + @Override public Set getRestrictions(ColumnMetadata columnDef) { return restrictions.getRestrictions(columnDef); } - - @Override - public String toString() - { - return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE); - } } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java index 77e0dd92fe64..8a016835fffb 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java @@ -52,18 +52,28 @@ public interface Restrictions extends Restriction * Checks if any of the underlying restriction is an IN. * @return true if any of the underlying restriction is an IN, false otherwise */ - public boolean hasIN(); + default public boolean hasIN() + { + return false; + } /** * Checks if any of the underlying restrictions is a CONTAINS / CONTAINS KEY restriction. * @return true if any of the underlying restrictions is CONTAINS, false otherwise */ - public boolean hasContains(); + default public boolean hasContains() + { + return false; + } + /** * Checks if any of the underlying restrictions is a slice. * @return true if any of the underlying restrictions is a slice, false otherwise */ - public boolean hasSlice(); + default public boolean hasSlice() + { + return false; + } /** * Checks if all of the underlying restrictions are EQ or IN restrictions. @@ -71,5 +81,8 @@ public interface Restrictions extends Restriction * @return true if all of the underlying restrictions are EQ or IN restrictions, * false otherwise */ - public boolean hasOnlyEqualityRestrictions(); + default public boolean hasOnlyEqualityRestrictions() + { + return true; + } } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java index 1b3482b72b18..2a7e0f54111f 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java @@ -22,15 +22,17 @@ import java.util.Collections; import java.util.List; +import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.*; import org.apache.cassandra.cql3.Term.Terminal; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.cql3.statements.Bound; import org.apache.cassandra.db.MultiCBuilder; -import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.index.Index; import org.apache.cassandra.index.IndexRegistry; +import org.apache.cassandra.serializers.ListSerializer; +import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.Pair; @@ -80,6 +82,16 @@ public boolean hasSupportingIndex(IndexRegistry indexRegistry) return false; } + @Override + public boolean needsFiltering(Index.Group indexGroup) + { + for (Index index : indexGroup.getIndexes()) + if (isSupportedBy(index)) + return false; + + return true; + } + @Override public final SingleRestriction mergeWith(SingleRestriction otherRestriction) { @@ -150,7 +162,7 @@ MultiColumnRestriction toMultiColumnRestriction() } @Override - public void addRowFilterTo(RowFilter filter, + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { @@ -214,11 +226,18 @@ public MultiCBuilder appendTo(MultiCBuilder builder, QueryOptions options) } @Override - public void addRowFilterTo(RowFilter filter, + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { - throw invalidRequest("IN restrictions are not supported on indexed columns"); + List values = getValues(options); + for (ByteBuffer v : values) + { + checkNotNull(v, "Invalid null value for column %s", columnDef.name); + checkBindValueSet(v, "Invalid unset value for column %s", columnDef.name); + } + ByteBuffer buffer = ListSerializer.pack(values, values.size(), ProtocolVersion.V3); + filter.add(columnDef, Operator.IN, buffer); } @Override @@ -385,7 +404,7 @@ public SingleRestriction doMergeWith(SingleRestriction otherRestriction) } @Override - public void addRowFilterTo(RowFilter filter, IndexRegistry indexRegistry, QueryOptions options) + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { for (Bound b : Bound.values()) if (hasBound(b)) @@ -475,7 +494,7 @@ public SingleRestriction doMergeWith(SingleRestriction otherRestriction) } @Override - public void addRowFilterTo(RowFilter filter, IndexRegistry indexRegistry, QueryOptions options) + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { for (ByteBuffer value : bindAndGet(values, options)) filter.add(columnDef, Operator.CONTAINS, value); @@ -614,7 +633,7 @@ MultiColumnRestriction toMultiColumnRestriction() } @Override - public void addRowFilterTo(RowFilter filter, + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { @@ -690,7 +709,7 @@ MultiColumnRestriction toMultiColumnRestriction() } @Override - public void addRowFilterTo(RowFilter filter, + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java index d1837d459ddb..61cdc2287173 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java @@ -21,6 +21,9 @@ import java.util.*; import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import org.apache.cassandra.cql3.*; import org.apache.cassandra.cql3.functions.Function; @@ -35,11 +38,9 @@ import org.apache.cassandra.index.IndexRegistry; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.utils.btree.BTreeSet; -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; - 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.invalidRequest; @@ -47,17 +48,28 @@ /** * The restrictions corresponding to the relations specified on the where-clause of CQL query. */ -public final class StatementRestrictions +public class StatementRestrictions { public static final String REQUIRES_ALLOW_FILTERING_MESSAGE = - "Cannot execute this query as it might involve data filtering and " + - "thus may have unpredictable performance. If you want to execute " + - "this query despite the performance unpredictability, use ALLOW FILTERING"; + "Cannot execute this query as it might involve data filtering and " + + "thus may have unpredictable performance. If you want to execute " + + "this query despite the performance unpredictability, use ALLOW FILTERING"; - /** - * The type of statement - */ - private final StatementType type; + public static final String HAS_UNSUPPORTED_INDEX_RESTRICTION_MESSAGE_SINGLE = + "Column '%s' has an index but does not support the operators specified in the query. " + + "If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING"; + + public static final String HAS_UNSUPPORTED_INDEX_RESTRICTION_MESSAGE_MULTI = + "Columns %s have indexes but do not support the operators specified in the query. " + + "If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING"; + + public static final String INDEX_DOES_NOT_SUPPORT_LIKE_MESSAGE = "Index on column %s does not support LIKE restrictions."; + + public static final String INDEX_DOES_NOT_SUPPORT_DISJUNCTION = + "An index involved in this query does not support disjunctive queries using the OR operator"; + + public static final String PARTITION_KEY_RESTRICTION_MUST_BE_TOP_LEVEL = + "Restriction on partition key column %s must not be nested under OR operator"; /** * The Column Family meta data @@ -67,34 +79,39 @@ public final class StatementRestrictions /** * Restrictions on partitioning columns */ - private PartitionKeyRestrictions partitionKeyRestrictions; + protected final PartitionKeyRestrictions partitionKeyRestrictions; /** * Restrictions on clustering columns */ - private ClusteringColumnRestrictions clusteringColumnsRestrictions; + private final ClusteringColumnRestrictions clusteringColumnsRestrictions; /** * Restriction on non-primary key columns (i.e. secondary index restrictions) */ - private RestrictionSet nonPrimaryKeyRestrictions; + private final RestrictionSet nonPrimaryKeyRestrictions; - private Set notNullColumns; + private final ImmutableSet notNullColumns; /** * The restrictions used to build the row filter */ - private final IndexRestrictions filterRestrictions = new IndexRestrictions(); + private final IndexRestrictions filterRestrictions; + + /** + * true if these restrictions form part of an OR query, false otherwise + */ + private boolean isDisjunction; /** * true if the secondary index need to be queried, false otherwise */ - private boolean usesSecondaryIndexing; + protected boolean usesSecondaryIndexing; /** * Specify if the query will return a range of partition keys. */ - private boolean isKeyRange; + protected boolean isKeyRange; /** * true if nonPrimaryKeyRestrictions contains restriction on a regular column, @@ -102,182 +119,594 @@ public final class StatementRestrictions */ private boolean hasRegularColumnsRestrictions; + private final List children; + /** * Creates a new empty StatementRestrictions. * - * @param type the type of statement * @param table the column family meta data * @return a new empty StatementRestrictions. */ - public static StatementRestrictions empty(StatementType type, TableMetadata table) + public static StatementRestrictions empty(TableMetadata table) { - return new StatementRestrictions(type, table, false); + return new StatementRestrictions(table, false); } - private StatementRestrictions(StatementType type, TableMetadata table, boolean allowFiltering) + private StatementRestrictions(TableMetadata table, boolean allowFiltering) { - this.type = type; this.table = table; - this.partitionKeyRestrictions = new PartitionKeySingleRestrictionSet(table.partitionKeyAsClusteringComparator()); - this.clusteringColumnsRestrictions = new ClusteringColumnRestrictions(table, allowFiltering); - this.nonPrimaryKeyRestrictions = new RestrictionSet(); - this.notNullColumns = new HashSet<>(); + this.partitionKeyRestrictions = PartitionKeySingleRestrictionSet.builder(table.partitionKeyAsClusteringComparator()).build(); + this.clusteringColumnsRestrictions = ClusteringColumnRestrictions.builder(table, allowFiltering).build(); + this.nonPrimaryKeyRestrictions = RestrictionSet.builder().build(); + this.notNullColumns = ImmutableSet.of(); + this.filterRestrictions = IndexRestrictions.of(); + this.children = Collections.emptyList(); } - public StatementRestrictions(StatementType type, - TableMetadata table, - WhereClause whereClause, - VariableSpecifications boundNames, - boolean selectsOnlyStaticColumns, - boolean allowFiltering, - boolean forView) + /** + * Adds the following restrictions to the index restrictions. + * + * @param restrictions the restrictions to add to the index restrictions + * @return a new {@code StatementRestrictions} with the new index restrictions + */ + public StatementRestrictions addIndexRestrictions(Restrictions restrictions) { - this(type, table, whereClause, boundNames, selectsOnlyStaticColumns, type.allowUseOfSecondaryIndices(), allowFiltering, forView); + IndexRestrictions newIndexRestrictions = IndexRestrictions.builder() + .add(filterRestrictions) + .add(restrictions) + .build(); + + return new StatementRestrictions(table, + partitionKeyRestrictions, + clusteringColumnsRestrictions, + nonPrimaryKeyRestrictions, + notNullColumns, + isDisjunction, + usesSecondaryIndexing, + isKeyRange, + hasRegularColumnsRestrictions, + newIndexRestrictions, + children); } - /* - * We want to override allowUseOfSecondaryIndices flag from the StatementType for MV statements - * to avoid initing the Keyspace and SecondaryIndexManager. + /** + * Adds the following external restrictions (mostly custom and user index expressions) to the index restrictions. + * + * @param restrictions the restrictions to add to the index restrictions + * @return a new {@code StatementRestrictions} with the new index restrictions */ - public StatementRestrictions(StatementType type, - TableMetadata table, - WhereClause whereClause, - VariableSpecifications boundNames, - boolean selectsOnlyStaticColumns, - boolean allowUseOfSecondaryIndices, - boolean allowFiltering, - boolean forView) + public StatementRestrictions addExternalRestrictions(Iterable restrictions) { - this(type, table, allowFiltering); + IndexRestrictions.Builder newIndexRestrictions = IndexRestrictions.builder() + .add(filterRestrictions); + + for (CustomIndexExpression restriction : restrictions) + newIndexRestrictions.add(restriction); + + return new StatementRestrictions(table, + partitionKeyRestrictions, + clusteringColumnsRestrictions, + nonPrimaryKeyRestrictions, + notNullColumns, + isDisjunction, + usesSecondaryIndexing, + isKeyRange, + hasRegularColumnsRestrictions, + newIndexRestrictions.build(), + children); + } - IndexRegistry indexRegistry = null; - if (type.allowUseOfSecondaryIndices()) - indexRegistry = IndexRegistry.obtain(table); + private StatementRestrictions(TableMetadata table, + PartitionKeyRestrictions partitionKeyRestrictions, + ClusteringColumnRestrictions clusteringColumnsRestrictions, + RestrictionSet nonPrimaryKeyRestrictions, + ImmutableSet notNullColumns, + boolean isDisjunction, + boolean usesSecondaryIndexing, + boolean isKeyRange, + boolean hasRegularColumnsRestrictions, + IndexRestrictions filterRestrictions, + List children) + { + this.table = table; + this.partitionKeyRestrictions = partitionKeyRestrictions; + this.clusteringColumnsRestrictions = clusteringColumnsRestrictions; + this.nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions; + this.notNullColumns = notNullColumns; + this.filterRestrictions = filterRestrictions; + this.isDisjunction = isDisjunction; + this.usesSecondaryIndexing = usesSecondaryIndexing; + this.isKeyRange = isKeyRange; + this.hasRegularColumnsRestrictions = hasRegularColumnsRestrictions; + this.children = children; + } - /* - * WHERE clause. For a given entity, rules are: - * - EQ relation conflicts with anything else (including a 2nd EQ) - * - Can't have more than one LT(E) relation (resp. GT(E) relation) - * - IN relation are restricted to row keys (for now) and conflicts with anything else (we could - * allow two IN for the same entity but that doesn't seem very useful) - * - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value - * in CQL so far) + public static StatementRestrictions create(StatementType type, + TableMetadata table, + WhereClause whereClause, + VariableSpecifications boundNames, + boolean selectsOnlyStaticColumns, + boolean allowFiltering, + boolean forView) + { + return new Builder(type, + table, + whereClause, + boundNames, + selectsOnlyStaticColumns, + type.allowUseOfSecondaryIndices(), + allowFiltering, + forView).build(); + } + + public static StatementRestrictions create(StatementType type, + TableMetadata table, + WhereClause whereClause, + VariableSpecifications boundNames, + boolean selectsOnlyStaticColumns, + boolean allowUseOfSecondaryIndices, + boolean allowFiltering, + boolean forView) + { + return new Builder(type, + table, + whereClause, + boundNames, + selectsOnlyStaticColumns, + allowUseOfSecondaryIndices, + allowFiltering, + forView).build(); + } + + /** + * Build a StatementRestrictions from a WhereClause for a given + * StatementType, TableMetadata and VariableSpecifications + * + * The validation rules for whether the StatementRestrictions are valid depend on a + * number of considerations, including whether indexes are being used and whether filtering is being + * used. + */ + public static class Builder + { + private final StatementType type; + private final TableMetadata table; + private final WhereClause whereClause; + private final VariableSpecifications boundNames; + private final boolean selectsOnlyStaticColumns; + private final boolean allowUseOfSecondaryIndices; + private final boolean allowFiltering; + private final boolean forView; + + public Builder(StatementType type, + TableMetadata table, + WhereClause whereClause, + VariableSpecifications boundNames, + boolean selectsOnlyStaticColumns, + boolean allowUseOfSecondaryIndices, + boolean allowFiltering, + boolean forView) + { + this.type = type; + this.table = table; + this.whereClause = whereClause; + this.boundNames = boundNames; + this.selectsOnlyStaticColumns = selectsOnlyStaticColumns; + this.allowUseOfSecondaryIndices = allowUseOfSecondaryIndices; + this.allowFiltering = allowFiltering; + this.forView = forView; + } + + public StatementRestrictions build() + { + IndexRegistry indexRegistry = null; + + // We want to avoid opening the keyspace during view construction + // since we're parsing these for restore and the base table or keyspace might not exist in the current schema. + if (allowUseOfSecondaryIndices && type.allowUseOfSecondaryIndices()) + indexRegistry = IndexRegistry.obtain(table); + + + WhereClause.AndElement root = whereClause.root().conjunctiveForm(); + return doBuild(root, indexRegistry, 0); + } + + /** + * Processes the WHERE clause expression tree recursively and assigns the restrictions to different sets + * based on the columns they are applied to. + * + * @param element root of the tree + * @param nestingLevel recursion depth needed to reject the restrictions that + * are not allowed to be nested (e.g. partition key restrictions) */ - for (Relation relation : whereClause.relations) + StatementRestrictions doBuild(WhereClause.ExpressionElement element, IndexRegistry indexRegistry, int nestingLevel) { - if (relation.operator() == Operator.IS_NOT) + assert element instanceof WhereClause.AndElement || nestingLevel > 0: + "Root of the WHERE clause expression tree must be a conjunction"; + + PartitionKeySingleRestrictionSet.Builder partitionKeyRestrictionSet = PartitionKeySingleRestrictionSet.builder(table.partitionKeyAsClusteringComparator()); + ClusteringColumnRestrictions.Builder clusteringColumnsRestrictionSet = ClusteringColumnRestrictions.builder(table, allowFiltering, indexRegistry); + RestrictionSet.Builder nonPrimaryKeyRestrictionSet = RestrictionSet.builder(); + ImmutableSet.Builder notNullColumnsBuilder = ImmutableSet.builder(); + + for (Relation relation : element.relations()) + { + if (relation.operator() == Operator.IS_NOT) + { + if (!forView) + throw invalidRequest("Unsupported restriction: %s", relation); + + notNullColumnsBuilder.addAll(relation.toRestriction(table, boundNames).getColumnDefs()); + } + else + { + Restriction restriction = relation.toRestriction(table, boundNames); + + if (relation.isLIKE() && (!type.allowUseOfSecondaryIndices() || !restriction.hasSupportingIndex(indexRegistry))) + { + if (getColumnsWithUnsupportedIndexRestrictions(table, ImmutableList.of(restriction)).isEmpty()) + { + throw invalidRequest("LIKE restriction is only supported on properly indexed columns. %s is not valid.", relation.toString()); + } + else + { + throw invalidRequest(StatementRestrictions.INDEX_DOES_NOT_SUPPORT_LIKE_MESSAGE, restriction.getFirstColumn()); + } + } + + ColumnMetadata def = restriction.getFirstColumn(); + if (def.isPartitionKey()) + { + // All partition key restrictions must be a part of the top-level AND operation. + // The read path filtering logic is currently unable to filter rows based on + // partition key restriction that is a part of a complex expression involving disjunctions. + // ALLOW FILTERING does not cut it, as RowFilter can't handle ORed partition + // key restrictions properly. + if (nestingLevel > 0) + throw invalidRequest(StatementRestrictions.PARTITION_KEY_RESTRICTION_MUST_BE_TOP_LEVEL, def); + + partitionKeyRestrictionSet.addRestriction(restriction); + } + // If a clustering column restriction is nested (under OR operator), + // we can't treat it as a real clustering column, + // but instead we treat it as a regular column and use + // index (if we have one) or use row filtering on it; hence we require nestingLevel == 0 check here + else if (def.isClusteringColumn() && nestingLevel == 0) + { + clusteringColumnsRestrictionSet.addRestriction(restriction); + } + else + { + nonPrimaryKeyRestrictionSet.addRestriction((SingleRestriction) restriction, element.isDisjunction()); + } + } + } + + PartitionKeyRestrictions partitionKeyRestrictions = partitionKeyRestrictionSet.build(); + ClusteringColumnRestrictions clusteringColumnsRestrictions = clusteringColumnsRestrictionSet.build(); + RestrictionSet nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictionSet.build(); + ImmutableSet notNullColumns = notNullColumnsBuilder.build(); + boolean hasRegularColumnsRestrictions = nonPrimaryKeyRestrictions.hasRestrictionFor(ColumnMetadata.Kind.REGULAR); + boolean usesSecondaryIndexing = false; + boolean isKeyRange = false; + + boolean hasQueriableClusteringColumnIndex = false; + boolean hasQueriableIndex = false; + + IndexRestrictions.Builder filterRestrictionsBuilder = IndexRestrictions.builder(); + + if (allowUseOfSecondaryIndices) { - if (!forView) - throw new InvalidRequestException("Unsupported restriction: " + relation); + if (element.containsCustomExpressions()) + { + CustomIndexExpression customExpression = prepareCustomIndexExpression(element.expressions(), + boundNames, + indexRegistry); + filterRestrictionsBuilder.add(customExpression); + } - this.notNullColumns.addAll(relation.toRestriction(table, boundNames).getColumnDefs()); + hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(indexRegistry); + hasQueriableIndex = element.containsCustomExpressions() + || hasQueriableClusteringColumnIndex + || partitionKeyRestrictions.hasSupportingIndex(indexRegistry) + || nonPrimaryKeyRestrictions.hasSupportingIndex(indexRegistry); } - else if (relation.isLIKE()) + + // At this point, the select statement if fully constructed, but we still have a few things to validate + if (!type.allowPartitionKeyRanges()) { - Restriction restriction = relation.toRestriction(table, boundNames); + checkFalse(partitionKeyRestrictions.isOnToken(), + "The token function cannot be used in WHERE clauses for %s statements", type); - if (!type.allowUseOfSecondaryIndices() || !restriction.hasSupportingIndex(indexRegistry)) - throw new InvalidRequestException(String.format("LIKE restriction is only supported on properly " + - "indexed columns. %s is not valid.", - relation.toString())); + if (partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table)) + throw invalidRequest("Some partition key parts are missing: %s", + Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents(partitionKeyRestrictions))); - addRestriction(restriction); + // slice query + checkFalse(partitionKeyRestrictions.hasSlice(), + "Only EQ and IN relation are supported on the partition key (unless you use the token() function)" + + " for %s statements", type); } else { - addRestriction(relation.toRestriction(table, boundNames)); + // If there are no partition restrictions or there's only token restriction, we have to set a key range + if (partitionKeyRestrictions.isOnToken()) + isKeyRange = true; + + if (partitionKeyRestrictions.isEmpty() && partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table)) + { + isKeyRange = true; + usesSecondaryIndexing = hasQueriableIndex; + } + + // If there is a queriable index, no special condition is required on the other restrictions. + // But we still need to know 2 things: + // - If we don't have a queriable index, is the query ok + // - Is it queriable without 2ndary index, which is always more efficient + // If a component of the partition key is restricted by a relation, all preceding + // components must have a EQ. Only the last partition key component can be in IN relation. + // If partition key restrictions exist and this is a disjunction then we may need filtering + if (partitionKeyRestrictions.needFiltering(table) || (!partitionKeyRestrictions.isEmpty() && element.isDisjunction())) + { + if (!allowFiltering && !forView && !hasQueriableIndex) + throw new InvalidRequestException(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); + + isKeyRange = true; + usesSecondaryIndexing = hasQueriableIndex; + } } - } - hasRegularColumnsRestrictions = nonPrimaryKeyRestrictions.hasRestrictionFor(ColumnMetadata.Kind.REGULAR); + // Some but not all of the partition key columns have been specified or they form part of a disjunction; + // hence we need turn these restrictions into a row filter. + if (usesSecondaryIndexing || partitionKeyRestrictions.needFiltering(table) || element.isDisjunction()) + filterRestrictionsBuilder.add(partitionKeyRestrictions); - boolean hasQueriableClusteringColumnIndex = false; - boolean hasQueriableIndex = false; + if (selectsOnlyStaticColumns && !clusteringColumnsRestrictions.isEmpty()) + { + // If the only updated/deleted columns are static, then we don't need clustering columns. + // And in fact, unless it is an INSERT, we reject if clustering colums are provided as that + // suggest something unintended. For instance, given: + // CREATE TABLE t (k int, v int, s int static, PRIMARY KEY (k, v)) + // it can make sense to do: + // INSERT INTO t(k, v, s) VALUES (0, 1, 2) + // but both + // UPDATE t SET s = 3 WHERE k = 0 AND v = 1 + // DELETE v FROM t WHERE k = 0 AND v = 1 + // sounds like you don't really understand what your are doing. + if (type.isDelete() || type.isUpdate()) + throw invalidRequest("Invalid restrictions on clustering columns since the %s statement modifies only static columns", + type); + } - if (allowUseOfSecondaryIndices) - { - if (whereClause.containsCustomExpressions()) - processCustomIndexExpressions(whereClause.expressions, boundNames, indexRegistry); - - hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(indexRegistry); - hasQueriableIndex = !filterRestrictions.getCustomIndexExpressions().isEmpty() - || hasQueriableClusteringColumnIndex - || partitionKeyRestrictions.hasSupportingIndex(indexRegistry) - || nonPrimaryKeyRestrictions.hasSupportingIndex(indexRegistry); - } + // Now process and validate the clustering column restrictions + checkFalse(!type.allowClusteringColumnSlices() && clusteringColumnsRestrictions.hasSlice(), + "Slice restrictions are not supported on the clustering columns in %s statements", type); - // At this point, the select statement if fully constructed, but we still have a few things to validate - processPartitionKeyRestrictions(hasQueriableIndex, allowFiltering, forView); + if (!type.allowClusteringColumnSlices() + && (!table.isCompactTable() || (table.isCompactTable() && clusteringColumnsRestrictions.isEmpty()))) + { + if (!selectsOnlyStaticColumns && (table.clusteringColumns().size() != clusteringColumnsRestrictions.size())) + throw invalidRequest("Some clustering keys are missing: %s", + Joiner.on(", ").join(getUnrestrictedClusteringColumns(clusteringColumnsRestrictions))); + } + else + { + checkFalse(clusteringColumnsRestrictions.hasContains() && !hasQueriableIndex && !allowFiltering, + "Clustering columns can only be restricted with CONTAINS with a secondary index or filtering"); - // Some but not all of the partition key columns have been specified; - // hence we need turn these restrictions into a row filter. - if (usesSecondaryIndexing || partitionKeyRestrictions.needFiltering(table)) - filterRestrictions.add(partitionKeyRestrictions); + if (!clusteringColumnsRestrictions.isEmpty() && clusteringColumnsRestrictions.needFiltering()) + { + if (hasQueriableIndex || forView) + { + usesSecondaryIndexing = true; + } + else if (!allowFiltering) + { + List clusteringColumns = table.clusteringColumns(); + List restrictedColumns = clusteringColumnsRestrictions.getColumnDefs(); - if (selectsOnlyStaticColumns && hasClusteringColumnsRestrictions()) + for (int i = 0, m = restrictedColumns.size(); i < m; i++) + { + ColumnMetadata clusteringColumn = clusteringColumns.get(i); + ColumnMetadata restrictedColumn = restrictedColumns.get(i); + + if (!clusteringColumn.equals(restrictedColumn)) + { + throw invalidRequest("PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted", + restrictedColumn.name, + clusteringColumn.name); + } + } + } + } + } + + // Covers indexes on the first clustering column (among others). + if (isKeyRange && hasQueriableClusteringColumnIndex) + usesSecondaryIndexing = true; + + if (usesSecondaryIndexing || clusteringColumnsRestrictions.needFiltering()) + filterRestrictionsBuilder.add(clusteringColumnsRestrictions); + + // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if + // there is restrictions not covered by the PK. + if (!nonPrimaryKeyRestrictions.isEmpty()) + { + if (!type.allowNonPrimaryKeyInWhereClause()) + { + Collection nonPrimaryKeyColumns = + ColumnMetadata.toIdentifiers(nonPrimaryKeyRestrictions.getColumnDefs()); + + throw invalidRequest("Non PRIMARY KEY columns found in where clause: %s ", + Joiner.on(", ").join(nonPrimaryKeyColumns)); + } + if (hasQueriableIndex) + usesSecondaryIndexing = true; + else if (!allowFiltering) + throwRequiresAllowFilteringError(table, clusteringColumnsRestrictions, nonPrimaryKeyRestrictions); + + filterRestrictionsBuilder.add(nonPrimaryKeyRestrictions); + } + + if (usesSecondaryIndexing) + checkFalse(partitionKeyRestrictions.hasIN(), + "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported"); + + ImmutableList.Builder children = ImmutableList.builder(); + + for (WhereClause.ContainerElement container : element.operations()) + children.add(doBuild(container, indexRegistry, nestingLevel + 1)); + + return new StatementRestrictions(table, + partitionKeyRestrictions, + clusteringColumnsRestrictions, + nonPrimaryKeyRestrictions, + notNullColumns, + element.isDisjunction(), + usesSecondaryIndexing, + isKeyRange, + hasRegularColumnsRestrictions, + filterRestrictionsBuilder.build(), + children.build()); + } + + private Set getColumnsWithUnsupportedIndexRestrictions(TableMetadata table, + ClusteringColumnRestrictions clusteringColumnsRestrictions, + RestrictionSet nonPrimaryKeyRestrictions) { - // If the only updated/deleted columns are static, then we don't need clustering columns. - // And in fact, unless it is an INSERT, we reject if clustering colums are provided as that - // suggest something unintended. For instance, given: - // CREATE TABLE t (k int, v int, s int static, PRIMARY KEY (k, v)) - // it can make sense to do: - // INSERT INTO t(k, v, s) VALUES (0, 1, 2) - // but both - // UPDATE t SET s = 3 WHERE k = 0 AND v = 1 - // DELETE v FROM t WHERE k = 0 AND v = 1 - // sounds like you don't really understand what your are doing. - if (type.isDelete() || type.isUpdate()) - throw invalidRequest("Invalid restrictions on clustering columns since the %s statement modifies only static columns", - type); - if (type.isSelect()) - throw invalidRequest("Cannot restrict clustering columns when selecting only static columns"); + return getColumnsWithUnsupportedIndexRestrictions(table, Iterables.concat(clusteringColumnsRestrictions.restrictions(), nonPrimaryKeyRestrictions.restrictions())); } - processClusteringColumnsRestrictions(hasQueriableIndex, - selectsOnlyStaticColumns, - forView, - allowFiltering); + private Set getColumnsWithUnsupportedIndexRestrictions(TableMetadata table, Iterable restrictions) + { + IndexRegistry indexRegistry = IndexRegistry.obtain(table); + if (indexRegistry.listIndexes().isEmpty()) + return Collections.emptySet(); + + ImmutableSet.Builder builder = ImmutableSet.builder(); - // Covers indexes on the first clustering column (among others). - if (isKeyRange && hasQueriableClusteringColumnIndex) - usesSecondaryIndexing = true; + for (Restriction restriction : restrictions) + { + if (!restriction.hasSupportingIndex(indexRegistry)) + { + for (Index index : indexRegistry.listIndexes()) + { + // If a column restriction has an index which was not picked up by hasSupportingIndex, it means it's an unsupported restriction + for (ColumnMetadata column : restriction.getColumnDefs()) + { + if (index.dependsOn(column)) + builder.add(column); + } + } + } + } - if (usesSecondaryIndexing || clusteringColumnsRestrictions.needFiltering()) - filterRestrictions.add(clusteringColumnsRestrictions); + return builder.build(); + } - // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if - // there is restrictions not covered by the PK. - if (!nonPrimaryKeyRestrictions.isEmpty()) + private void throwRequiresAllowFilteringError(TableMetadata table, + ClusteringColumnRestrictions clusteringColumnsRestrictions, + RestrictionSet nonPrimaryKeyRestrictions) { - if (!type.allowNonPrimaryKeyInWhereClause()) + Set unsupported = getColumnsWithUnsupportedIndexRestrictions(table, + clusteringColumnsRestrictions, + nonPrimaryKeyRestrictions); + if (unsupported.isEmpty()) { - Collection nonPrimaryKeyColumns = - ColumnMetadata.toIdentifiers(nonPrimaryKeyRestrictions.getColumnDefs()); - - throw invalidRequest("Non PRIMARY KEY columns found in where clause: %s ", - Joiner.on(", ").join(nonPrimaryKeyColumns)); - } - if (hasQueriableIndex) - usesSecondaryIndexing = true; - else if (!allowFiltering) throw invalidRequest(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); + } + else + { + // If there's an index on these columns but the restriction is not supported on this index, throw a more specific error message + if (unsupported.size() == 1) + throw invalidRequest(String.format(StatementRestrictions.HAS_UNSUPPORTED_INDEX_RESTRICTION_MESSAGE_SINGLE, unsupported.iterator().next())); + else + throw invalidRequest(String.format(StatementRestrictions.HAS_UNSUPPORTED_INDEX_RESTRICTION_MESSAGE_MULTI, unsupported)); + } + } + + + private CustomIndexExpression prepareCustomIndexExpression(List expressions, + VariableSpecifications boundNames, + IndexRegistry indexRegistry) + { + if (expressions.size() > 1) + throw new InvalidRequestException(IndexRestrictions.MULTIPLE_EXPRESSIONS); + + CustomIndexExpression expression = expressions.get(0); + + QualifiedName name = expression.targetIndex; + + if (name.hasKeyspace() && !name.getKeyspace().equals(table.keyspace)) + throw IndexRestrictions.invalidIndex(expression.targetIndex, table); - filterRestrictions.add(nonPrimaryKeyRestrictions); + if (!table.indexes.has(expression.targetIndex.getName())) + throw IndexRestrictions.indexNotFound(expression.targetIndex, table); + + Index index = indexRegistry.getIndex(table.indexes.get(expression.targetIndex.getName()).get()); + if (!index.getIndexMetadata().isCustom()) + throw IndexRestrictions.nonCustomIndexInExpression(expression.targetIndex); + + AbstractType expressionType = index.customExpressionValueType(); + if (expressionType == null) + throw IndexRestrictions.customExpressionNotSupported(expression.targetIndex); + + expression.prepareValue(table, expressionType, boundNames); + return expression; } - if (usesSecondaryIndexing) - validateSecondaryIndexSelections(); + /** + * Returns the partition key components that are not restricted. + * @return the partition key components that are not restricted. + */ + private Collection getPartitionKeyUnrestrictedComponents(PartitionKeyRestrictions partitionKeyRestrictions) + { + List list = new ArrayList<>(table.partitionKeyColumns()); + list.removeAll(partitionKeyRestrictions.getColumnDefs()); + return ColumnMetadata.toIdentifiers(list); + } + + /** + * Returns the clustering columns that are not restricted. + * @return the clustering columns that are not restricted. + */ + private Collection getUnrestrictedClusteringColumns(ClusteringColumnRestrictions clusteringColumnsRestrictions) + { + List missingClusteringColumns = new ArrayList<>(table.clusteringColumns()); + missingClusteringColumns.removeAll(clusteringColumnsRestrictions.getColumnDefs()); + return ColumnMetadata.toIdentifiers(missingClusteringColumns); + } + } + + public IndexRestrictions filterRestrictions() + { + return filterRestrictions; + } + + public List children() + { + return children; } - private void addRestriction(Restriction restriction) + public void throwRequiresAllowFilteringError(TableMetadata table) { - ColumnMetadata def = restriction.getFirstColumn(); - if (def.isPartitionKey()) - partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction); - else if (def.isClusteringColumn()) - clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction); + Set unsupported = getColumnsWithUnsupportedIndexRestrictions(table); + if (unsupported.isEmpty()) + { + throw invalidRequest(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); + } else - nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction((SingleRestriction) restriction); + { + // If there's an index on these columns but the restriction is not supported on this index, throw a more specific error message + if (unsupported.size() == 1) + throw invalidRequest(String.format(StatementRestrictions.HAS_UNSUPPORTED_INDEX_RESTRICTION_MESSAGE_SINGLE, unsupported.iterator().next())); + else + throw invalidRequest(String.format(StatementRestrictions.HAS_UNSUPPORTED_INDEX_RESTRICTION_MESSAGE_MULTI, unsupported)); + } + } + + public void throwsRequiresIndexSupportingDisjunctionError(TableMetadata table) + { + throw invalidRequest(StatementRestrictions.INDEX_DOES_NOT_SUPPORT_DISJUNCTION); } public void addFunctionsTo(List functions) @@ -285,6 +714,9 @@ public void addFunctionsTo(List functions) partitionKeyRestrictions.addFunctionsTo(functions); clusteringColumnsRestrictions.addFunctionsTo(functions); nonPrimaryKeyRestrictions.addFunctionsTo(functions); + + for (StatementRestrictions child : children) + child.addFunctionsTo(functions); } // may be used by QueryHandler implementations @@ -310,20 +742,23 @@ public Set nonPKRestrictedColumns(boolean includeNotNullRestrict if (includeNotNullRestrictions) { - for (ColumnMetadata def : notNullColumns) + for (ColumnMetadata def : notNullColumns()) { if (!def.isPrimaryKeyColumn()) columns.add(def); } } + for (StatementRestrictions child : children) + columns.addAll(child.nonPKRestrictedColumns(includeNotNullRestrictions)); + return columns; } /** * @return the set of columns that have an IS NOT NULL restriction on them */ - public Set notNullColumns() + public ImmutableSet notNullColumns() { return notNullColumns; } @@ -333,10 +768,17 @@ public Set notNullColumns() */ public boolean isRestricted(ColumnMetadata column) { - if (notNullColumns.contains(column)) + if (notNullColumns().contains(column)) + return true; + + if (getRestrictions(column.kind).getColumnDefs().contains(column)) return true; - return getRestrictions(column.kind).getColumnDefs().contains(column); + for (StatementRestrictions child : children) + if (child.isRestricted(column)) + return true; + + return false; } /** @@ -357,7 +799,7 @@ public boolean keyIsInRelation() */ public boolean isKeyRange() { - return this.isKeyRange; + return isKeyRange; } /** @@ -381,7 +823,7 @@ public boolean isColumnRestrictedByEq(ColumnMetadata columnDef) * @param kind the column type * @return the Restrictions for the specified type of columns */ - private Restrictions getRestrictions(ColumnMetadata.Kind kind) + protected Restrictions getRestrictions(ColumnMetadata.Kind kind) { switch (kind) { @@ -398,55 +840,14 @@ private Restrictions getRestrictions(ColumnMetadata.Kind kind) */ public boolean usesSecondaryIndexing() { - return this.usesSecondaryIndexing; - } - - private void processPartitionKeyRestrictions(boolean hasQueriableIndex, boolean allowFiltering, boolean forView) - { - if (!type.allowPartitionKeyRanges()) - { - checkFalse(partitionKeyRestrictions.isOnToken(), - "The token function cannot be used in WHERE clauses for %s statements", type); - - if (partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table)) - throw invalidRequest("Some partition key parts are missing: %s", - Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents())); - - // slice query - checkFalse(partitionKeyRestrictions.hasSlice(), - "Only EQ and IN relation are supported on the partition key (unless you use the token() function)" - + " for %s statements", type); - } - else - { - // If there are no partition restrictions or there's only token restriction, we have to set a key range - if (partitionKeyRestrictions.isOnToken()) - isKeyRange = true; - - if (partitionKeyRestrictions.isEmpty() && partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table)) - { - isKeyRange = true; - usesSecondaryIndexing = hasQueriableIndex; - } - - // If there is a queriable index, no special condition is required on the other restrictions. - // But we still need to know 2 things: - // - If we don't have a queriable index, is the query ok - // - Is it queriable without 2ndary index, which is always more efficient - // If a component of the partition key is restricted by a relation, all preceding - // components must have a EQ. Only the last partition key component can be in IN relation. - if (partitionKeyRestrictions.needFiltering(table)) - { - if (!allowFiltering && !forView && !hasQueriableIndex) - throw new InvalidRequestException(REQUIRES_ALLOW_FILTERING_MESSAGE); + if (usesSecondaryIndexing) + return true; - if (partitionKeyRestrictions.hasIN()) - throw new InvalidRequestException("IN restrictions are not supported when the query involves filtering"); + for (StatementRestrictions child: children) + if (child.usesSecondaryIndexing) + return true; - isKeyRange = true; - usesSecondaryIndexing = hasQueriableIndex; - } - } + return false; } public boolean hasPartitionKeyRestrictions() @@ -463,17 +864,6 @@ public boolean hasNonPrimaryKeyRestrictions() return !nonPrimaryKeyRestrictions.isEmpty(); } - /** - * Returns the partition key components that are not restricted. - * @return the partition key components that are not restricted. - */ - private Collection getPartitionKeyUnrestrictedComponents() - { - List list = new ArrayList<>(table.partitionKeyColumns()); - list.removeAll(partitionKeyRestrictions.getColumnDefs()); - return ColumnMetadata.toIdentifiers(list); - } - /** * Checks if the restrictions on the partition key are token restrictions. * @@ -496,74 +886,6 @@ public boolean clusteringKeyRestrictionsHasIN() return clusteringColumnsRestrictions.hasIN(); } - /** - * Processes the clustering column restrictions. - * - * @param hasQueriableIndex true if some of the queried data are indexed, false otherwise - * @param selectsOnlyStaticColumns true if the selected or modified columns are all statics, - * false otherwise. - */ - private void processClusteringColumnsRestrictions(boolean hasQueriableIndex, - boolean selectsOnlyStaticColumns, - boolean forView, - boolean allowFiltering) - { - checkFalse(!type.allowClusteringColumnSlices() && clusteringColumnsRestrictions.hasSlice(), - "Slice restrictions are not supported on the clustering columns in %s statements", type); - - if (!type.allowClusteringColumnSlices() - && (!table.isCompactTable() || (table.isCompactTable() && !hasClusteringColumnsRestrictions()))) - { - if (!selectsOnlyStaticColumns && hasUnrestrictedClusteringColumns()) - throw invalidRequest("Some clustering keys are missing: %s", - Joiner.on(", ").join(getUnrestrictedClusteringColumns())); - } - else - { - checkFalse(clusteringColumnsRestrictions.hasContains() && !hasQueriableIndex && !allowFiltering, - "Clustering columns can only be restricted with CONTAINS with a secondary index or filtering"); - - if (hasClusteringColumnsRestrictions() && clusteringColumnsRestrictions.needFiltering()) - { - if (hasQueriableIndex || forView) - { - usesSecondaryIndexing = true; - } - else if (!allowFiltering) - { - List clusteringColumns = table.clusteringColumns(); - List restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs()); - - for (int i = 0, m = restrictedColumns.size(); i < m; i++) - { - ColumnMetadata clusteringColumn = clusteringColumns.get(i); - ColumnMetadata restrictedColumn = restrictedColumns.get(i); - - if (!clusteringColumn.equals(restrictedColumn)) - { - throw invalidRequest("PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted", - restrictedColumn.name, - clusteringColumn.name); - } - } - } - } - - } - - } - - /** - * Returns the clustering columns that are not restricted. - * @return the clustering columns that are not restricted. - */ - private Collection getUnrestrictedClusteringColumns() - { - List missingClusteringColumns = new ArrayList<>(table.clusteringColumns()); - missingClusteringColumns.removeAll(new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs())); - return ColumnMetadata.toIdentifiers(missingClusteringColumns); - } - /** * Checks if some clustering columns are not restricted. * @return true if some clustering columns are not restricted, false otherwise. @@ -573,60 +895,24 @@ private boolean hasUnrestrictedClusteringColumns() return table.clusteringColumns().size() != clusteringColumnsRestrictions.size(); } - private void processCustomIndexExpressions(List expressions, - VariableSpecifications boundNames, - IndexRegistry indexRegistry) + public RowFilter getRowFilter(IndexRegistry indexManager, QueryOptions options) { - if (expressions.size() > 1) - throw new InvalidRequestException(IndexRestrictions.MULTIPLE_EXPRESSIONS); - - CustomIndexExpression expression = expressions.get(0); - - QualifiedName name = expression.targetIndex; - - if (name.hasKeyspace() && !name.getKeyspace().equals(table.keyspace)) - throw IndexRestrictions.invalidIndex(expression.targetIndex, table); - - if (!table.indexes.has(expression.targetIndex.getName())) - throw IndexRestrictions.indexNotFound(expression.targetIndex, table); - - Index index = indexRegistry.getIndex(table.indexes.get(expression.targetIndex.getName()).get()); - if (!index.getIndexMetadata().isCustom()) - throw IndexRestrictions.nonCustomIndexInExpression(expression.targetIndex); - - AbstractType expressionType = index.customExpressionValueType(); - if (expressionType == null) - throw IndexRestrictions.customExpressionNotSupported(expression.targetIndex); - - expression.prepareValue(table, expressionType, boundNames); - - filterRestrictions.add(expression); - } - - public RowFilter getRowFilter(IndexRegistry indexRegistry, QueryOptions options) - { - if (filterRestrictions.isEmpty()) + if (filterRestrictions.isEmpty() && children.isEmpty()) return RowFilter.NONE; - RowFilter filter = RowFilter.create(); - for (Restrictions restrictions : filterRestrictions.getRestrictions()) - restrictions.addRowFilterTo(filter, indexRegistry, options); - - for (CustomIndexExpression expression : filterRestrictions.getCustomIndexExpressions()) - expression.addToRowFilter(filter, table, options); - - return filter; + return RowFilter.builder().buildFromRestrictions(this, indexManager, table, options); } /** * Returns the partition keys for which the data is requested. * * @param options the query options + * @param queryState the query state * @return the partition keys for which the data is requested. */ - public List getPartitionKeys(final QueryOptions options) + public List getPartitionKeys(final QueryOptions options, QueryState queryState) { - return partitionKeyRestrictions.values(options); + return partitionKeyRestrictions.values(options, queryState); } /** @@ -665,7 +951,7 @@ private AbstractBounds getPartitionKeyBounds(IPartitioner p, { // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the // first component of a composite partition key) queries that filter on the partition key. - if (partitionKeyRestrictions.needFiltering(table)) + if (partitionKeyRestrictions.needFiltering(table) || isDisjunction) return new Range<>(p.getMinimumToken().minKeyBound(), p.getMinimumToken().maxKeyBound()); ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options); @@ -680,13 +966,13 @@ private AbstractBounds getPartitionKeyBounds(IPartitioner p, if (partitionKeyRestrictions.isInclusive(Bound.START)) { return partitionKeyRestrictions.isInclusive(Bound.END) - ? new Bounds<>(startKey, finishKey) - : new IncludingExcludingBounds<>(startKey, finishKey); + ? new Bounds<>(startKey, finishKey) + : new IncludingExcludingBounds<>(startKey, finishKey); } return partitionKeyRestrictions.isInclusive(Bound.END) - ? new Range<>(startKey, finishKey) - : new ExcludingBounds<>(startKey, finishKey); + ? new Range<>(startKey, finishKey) + : new ExcludingBounds<>(startKey, finishKey); } private AbstractBounds getPartitionKeyBoundsForTokenRestrictions(IPartitioner p, @@ -710,7 +996,7 @@ private AbstractBounds getPartitionKeyBoundsForTokenRestricti */ int cmp = startToken.compareTo(endToken); if (!startToken.isMinimum() && !endToken.isMinimum() - && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd)))) + && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd)))) return null; PartitionPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound(); @@ -744,9 +1030,10 @@ public boolean hasClusteringColumnsRestrictions() * Returns the requested clustering columns. * * @param options the query options + * @param queryState the query state * @return the requested clustering columns */ - public NavigableSet> getClusteringColumns(QueryOptions options) + public NavigableSet> getClusteringColumns(QueryOptions options, QueryState queryState) { // If this is a names command and the table is a static compact one, then as far as CQL is concerned we have // only a single row which internally correspond to the static parts. In which case we want to return an empty @@ -754,7 +1041,7 @@ public NavigableSet> getClusteringColumns(QueryOptions options) if (table.isStaticCompactTable()) return BTreeSet.empty(table.comparator); - return clusteringColumnsRestrictions.valuesAsClustering(options); + return clusteringColumnsRestrictions.valuesAsClustering(options, queryState); } /** @@ -769,6 +1056,11 @@ public NavigableSet> getClusteringColumnsBounds(Bound b, Quer return clusteringColumnsRestrictions.boundsAsClustering(b, options); } + public boolean isDisjunction() + { + return isDisjunction; + } + /** * Checks if the query returns a range of columns. * @@ -776,40 +1068,86 @@ public NavigableSet> getClusteringColumnsBounds(Bound b, Quer */ public boolean isColumnRange() { - int numberOfClusteringColumns = table.clusteringColumns().size(); - if (table.isStaticCompactTable()) - { - // For static compact tables we want to ignore the fake clustering column (note that if we weren't special casing, - // this would mean a 'SELECT *' on a static compact table would query whole partitions, even though we'll only return - // the static part as far as CQL is concerned. This is thus mostly an optimization to use the query-by-name path). - numberOfClusteringColumns = 0; - } - + // For static compact tables we want to ignore the fake clustering column (note that if we weren't special casing, + // this would mean a 'SELECT *' on a static compact table would query whole partitions, even though we'll only return + // the static part as far as CQL is concerned. This is thus mostly an optimization to use the query-by-name path). + int numberOfClusteringColumns = table.isStaticCompactTable() ? 0 : table.clusteringColumns().size(); // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ or IN. return clusteringColumnsRestrictions.size() < numberOfClusteringColumns - || !clusteringColumnsRestrictions.hasOnlyEqualityRestrictions(); + || !clusteringColumnsRestrictions.hasOnlyEqualityRestrictions(); } /** - * Checks if the query need to use filtering. + * Checks if the query needs to use filtering. + * * @return true if the query need to use filtering, false otherwise. */ - public boolean needFiltering() + public boolean needFiltering(TableMetadata table) { - int numberOfRestrictions = filterRestrictions.getCustomIndexExpressions().size(); - for (Restrictions restrictions : filterRestrictions.getRestrictions()) - numberOfRestrictions += restrictions.size(); + IndexRegistry indexRegistry = IndexRegistry.obtain(table); + boolean hasClusteringColumnRestrictions = !clusteringColumnsRestrictions.isEmpty(); + boolean hasMultipleContains = nonPrimaryKeyRestrictions.hasMultipleContains(); + if (filterRestrictions.needFiltering(indexRegistry, hasClusteringColumnRestrictions, hasMultipleContains)) + return true; + + for (StatementRestrictions child : children) + if (child.needFiltering(table)) + return true; + + return false; + } + + public boolean needsDisjunctionSupport(TableMetadata table) + { + boolean containsDisjunction = isDisjunction || !children.isEmpty(); + + if (!containsDisjunction) + return false; + + IndexRegistry indexRegistry = IndexRegistry.obtain(table); + + for (Index.Group group : indexRegistry.listIndexGroups()) + if (filterRestrictions.indexBeingUsed(group) && !group.supportsDisjunction()) + return true; - return numberOfRestrictions > 1 - || (numberOfRestrictions == 0 && !clusteringColumnsRestrictions.isEmpty()) - || (numberOfRestrictions != 0 - && nonPrimaryKeyRestrictions.hasMultipleContains()); + for (StatementRestrictions child : children) + if (child.needsDisjunctionSupport(table)) + return true; + + return false; } - private void validateSecondaryIndexSelections() + private Set getColumnsWithUnsupportedIndexRestrictions(TableMetadata table) { - checkFalse(keyIsInRelation(), - "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported"); + return getColumnsWithUnsupportedIndexRestrictions(table, Iterables.concat(clusteringColumnsRestrictions.restrictions(), + nonPrimaryKeyRestrictions.restrictions())); + } + + private Set getColumnsWithUnsupportedIndexRestrictions(TableMetadata table, Iterable restrictions) + { + IndexRegistry indexRegistry = IndexRegistry.obtain(table); + if (indexRegistry.listIndexes().isEmpty()) + return Collections.emptySet(); + + ImmutableSet.Builder builder = ImmutableSet.builder(); + + for (Restriction restriction : restrictions) + { + if (!restriction.hasSupportingIndex(indexRegistry)) + { + for (Index index : indexRegistry.listIndexes()) + { + // If a column restriction has an index which was not picked up by hasSupportingIndex, it means it's an unsupported restriction + for (ColumnMetadata column : restriction.getColumnDefs()) + { + if (index.dependsOn(column)) + builder.add(column); + } + } + } + } + + return builder.build(); } /** @@ -821,10 +1159,10 @@ private void validateSecondaryIndexSelections() public boolean hasAllPKColumnsRestrictedByEqualities() { return !isPartitionKeyRestrictionsOnToken() - && !partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table) - && (partitionKeyRestrictions.hasOnlyEqualityRestrictions()) - && !hasUnrestrictedClusteringColumns() - && (clusteringColumnsRestrictions.hasOnlyEqualityRestrictions()); + && !partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table) + && (partitionKeyRestrictions.hasOnlyEqualityRestrictions()) + && !hasUnrestrictedClusteringColumns() + && (clusteringColumnsRestrictions.hasOnlyEqualityRestrictions()); } /** @@ -862,10 +1200,4 @@ public boolean returnStaticContentOnPartitionWithNoRows() // a full partition query, then we include that content. return queriesFullPartitions(); } - - @Override - public String toString() - { - return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE); - } } diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java index 437b17c617ab..8100777543e1 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java @@ -18,15 +18,16 @@ package org.apache.cassandra.cql3.restrictions; import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import com.google.common.collect.BoundType; import com.google.common.collect.ImmutableRangeSet; import com.google.common.collect.Range; import com.google.common.collect.RangeSet; -import org.apache.cassandra.schema.ColumnMetadata; -import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.cql3.statements.Bound; @@ -34,7 +35,11 @@ import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.Index; import org.apache.cassandra.index.IndexRegistry; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.QueryState; import static org.apache.cassandra.cql3.statements.Bound.END; import static org.apache.cassandra.cql3.statements.Bound.START; @@ -45,36 +50,108 @@ *

If all partition key columns have non-token restrictions and do not need filtering, they take precedence * when calculating bounds, incusiveness etc (see CASSANDRA-12149).

*/ -final class TokenFilter implements PartitionKeyRestrictions +abstract class TokenFilter implements PartitionKeyRestrictions { /** * The decorated restriction */ - private final PartitionKeyRestrictions restrictions; + final PartitionKeyRestrictions restrictions; /** * The restriction on the token */ - private final TokenRestriction tokenRestriction; + final TokenRestriction tokenRestriction; /** * Partitioner to manage tokens, extracted from tokenRestriction metadata. */ private final IPartitioner partitioner; - public boolean hasIN() + static TokenFilter create(PartitionKeyRestrictions restrictions, TokenRestriction tokenRestriction) { - return isOnToken() ? false : restrictions.hasIN(); + boolean onToken = restrictions.needFiltering(tokenRestriction.metadata) || restrictions.size() < tokenRestriction.size(); + return onToken ? new TokenFilter.OnToken(restrictions, tokenRestriction) + : new TokenFilter.NotOnToken(restrictions, tokenRestriction); } - public boolean hasContains() + private TokenFilter(PartitionKeyRestrictions restrictions, TokenRestriction tokenRestriction) { - return isOnToken() ? false : restrictions.hasContains(); + this.restrictions = restrictions; + this.tokenRestriction = tokenRestriction; + this.partitioner = tokenRestriction.metadata.partitioner; } - public boolean hasOnlyEqualityRestrictions() + private static final class OnToken extends TokenFilter { - return isOnToken() ? false : restrictions.hasOnlyEqualityRestrictions(); + private OnToken(PartitionKeyRestrictions restrictions, TokenRestriction tokenRestriction) + { + super(restrictions, tokenRestriction); + } + + @Override + public boolean isOnToken() + { + return true; + } + + @Override + public boolean isInclusive(Bound bound) + { + return tokenRestriction.isInclusive(bound); + } + + @Override + public boolean hasBound(Bound bound) + { + return tokenRestriction.hasBound(bound); + } + + @Override + public List bounds(Bound bound, QueryOptions options) throws InvalidRequestException + { + return tokenRestriction.bounds(bound, options); + } + } + + private static final class NotOnToken extends TokenFilter + { + private NotOnToken(PartitionKeyRestrictions restrictions, TokenRestriction tokenRestriction) + { + super(restrictions, tokenRestriction); + } + + @Override + public boolean isInclusive(Bound bound) + { + return restrictions.isInclusive(bound); + } + + @Override + public boolean hasBound(Bound bound) + { + return restrictions.hasBound(bound); + } + + @Override + public List bounds(Bound bound, QueryOptions options) throws InvalidRequestException + { + return restrictions.bounds(bound, options); + } + + public boolean hasIN() + { + return restrictions.hasIN(); + } + + public boolean hasContains() + { + return restrictions.hasContains(); + } + + public boolean hasOnlyEqualityRestrictions() + { + return restrictions.hasOnlyEqualityRestrictions(); + } } @Override @@ -94,44 +171,19 @@ public boolean isOnToken() return needFiltering(tokenRestriction.metadata) || restrictions.size() < tokenRestriction.size(); } - public TokenFilter(PartitionKeyRestrictions restrictions, TokenRestriction tokenRestriction) - { - this.restrictions = restrictions; - this.tokenRestriction = tokenRestriction; - this.partitioner = tokenRestriction.metadata.partitioner; - } - @Override - public List values(QueryOptions options) throws InvalidRequestException + public List values(QueryOptions options, QueryState queryState) throws InvalidRequestException { - return filter(restrictions.values(options), options); + return filter(restrictions.values(options, queryState), options, queryState); } @Override public PartitionKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException { if (restriction.isOnToken()) - return new TokenFilter(restrictions, (TokenRestriction) tokenRestriction.mergeWith(restriction)); + return TokenFilter.create(restrictions, (TokenRestriction) tokenRestriction.mergeWith(restriction)); - return new TokenFilter(restrictions.mergeWith(restriction), tokenRestriction); - } - - @Override - public boolean isInclusive(Bound bound) - { - return isOnToken() ? tokenRestriction.isInclusive(bound) : restrictions.isInclusive(bound); - } - - @Override - public boolean hasBound(Bound bound) - { - return isOnToken() ? tokenRestriction.hasBound(bound) : restrictions.hasBound(bound); - } - - @Override - public List bounds(Bound bound, QueryOptions options) throws InvalidRequestException - { - return isOnToken() ? tokenRestriction.bounds(bound, options) : restrictions.bounds(bound, options); + return TokenFilter.create(restrictions.mergeWith(restriction), tokenRestriction); } /** @@ -139,13 +191,14 @@ public List bounds(Bound bound, QueryOptions options) throws Invalid * * @param values the values returned by the decorated restriction * @param options the query options + * @param queryState the query state * @return the values matching the token restriction * @throws InvalidRequestException if the request is invalid */ - private List filter(List values, QueryOptions options) throws InvalidRequestException + private List filter(List values, QueryOptions options, QueryState queryState) throws InvalidRequestException { RangeSet rangeSet = tokenRestriction.hasSlice() ? toRangeSet(tokenRestriction, options) - : toRangeSet(tokenRestriction.values(options)); + : toRangeSet(tokenRestriction.values(options, queryState)); return filterWithRangeSet(rangeSet, values); } @@ -278,9 +331,15 @@ public boolean hasSupportingIndex(IndexRegistry indexRegistry) } @Override - public void addRowFilterTo(RowFilter filter, IndexRegistry indexRegistry, QueryOptions options) + public boolean needsFiltering(Index.Group indexGroup) + { + return restrictions.needsFiltering(indexGroup); + } + + @Override + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { - restrictions.addRowFilterTo(filter, indexRegistry, options); + restrictions.addToRowFilter(filter, indexRegistry, options); } @Override diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java index e71b17782d15..654f33bc7500 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java @@ -22,6 +22,7 @@ import com.google.common.base.Joiner; +import org.apache.cassandra.index.Index; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.QueryOptions; @@ -31,6 +32,7 @@ import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.index.IndexRegistry; +import org.apache.cassandra.service.QueryState; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; @@ -122,7 +124,13 @@ public boolean hasSupportingIndex(IndexRegistry indexRegistry) } @Override - public void addRowFilterTo(RowFilter filter, IndexRegistry indexRegistry, QueryOptions options) + public boolean needsFiltering(Index.Group indexGroup) + { + return false; + } + + @Override + public void addToRowFilter(RowFilter.Builder filter, IndexRegistry indexRegistry, QueryOptions options) { throw new UnsupportedOperationException("Index expression cannot be created for token restriction"); } @@ -153,7 +161,7 @@ protected final String getColumnNamesAsString() public final PartitionKeyRestrictions mergeWith(Restriction otherRestriction) throws InvalidRequestException { if (!otherRestriction.isOnToken()) - return new TokenFilter(toPartitionKeyRestrictions(otherRestriction), this); + return TokenFilter.create(toPartitionKeyRestrictions(otherRestriction), this); return doMergeWith((TokenRestriction) otherRestriction); } @@ -176,7 +184,9 @@ private PartitionKeyRestrictions toPartitionKeyRestrictions(Restriction restrict if (restriction instanceof PartitionKeyRestrictions) return (PartitionKeyRestrictions) restriction; - return new PartitionKeySingleRestrictionSet(metadata.partitionKeyAsClusteringComparator()).mergeWith(restriction); + return PartitionKeySingleRestrictionSet.builder(metadata.partitionKeyAsClusteringComparator()) + .addRestriction(restriction) + .build(); } public static final class EQRestriction extends TokenRestriction @@ -205,7 +215,10 @@ protected PartitionKeyRestrictions doMergeWith(TokenRestriction otherRestriction @Override public List bounds(Bound b, QueryOptions options) throws InvalidRequestException { - return values(options); + // QueryState is used by inSelectCartesianProduct guardrail to skip non-ordinary users. + // Passing null here to avoid polluting too many methods, because in case of EQ token restriction, + // it won't generate high cartesian product. + return values(options, null); } @Override @@ -221,7 +234,7 @@ public boolean isInclusive(Bound b) } @Override - public List values(QueryOptions options) throws InvalidRequestException + public List values(QueryOptions options, QueryState queryState) throws InvalidRequestException { return Collections.singletonList(value.bindAndGet(options)); } @@ -254,7 +267,7 @@ public boolean hasSlice() } @Override - public List values(QueryOptions options) throws InvalidRequestException + public List values(QueryOptions options, QueryState queryState) throws InvalidRequestException { throw new UnsupportedOperationException(); } diff --git a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java index c67fc0330e95..403d3bed2e20 100644 --- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java +++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java @@ -23,6 +23,7 @@ import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.db.marshal.UserType; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.transport.ProtocolVersion; @@ -89,7 +90,7 @@ public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidReque ByteBuffer value = selected.getOutput(protocolVersion); if (value == null) return null; - ByteBuffer[] buffers = type.split(value); + ByteBuffer[] buffers = type.split(ByteBufferAccessor.instance, value); return field < buffers.length ? buffers[field] : null; } diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java index de5360f52529..66759381ef5a 100644 --- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java +++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java @@ -114,6 +114,23 @@ default ColumnSpecification specForElementOrSlice(Selectable selected, ColumnSpe } } + /** + * Checks that this {@code Selectable} is or can be converted into the specified type. + * @param table the table schema + * @param type the expected type + * @throws InvalidRequestException if the {@code Selectable} can not be converted into the specified type + */ + default void validateType(TableMetadata table, AbstractType type) + { + ColumnSpecification receiver = new ColumnSpecification(table.keyspace, + table.name, + new ColumnIdentifier(toString(), true), + type); + + if (!testAssignment(table.keyspace, receiver).isAssignable()) + throw invalidRequest("%s is not of the expected type: %s", this, type.asCQL3Type()); + } + public interface Raw { public Selectable prepare(TableMetadata table); @@ -175,6 +192,8 @@ public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType type = expectedType; if (type == null) throw new InvalidRequestException("Cannot infer type for term " + this + " in selection clause (try using a cast to force a type)"); + + validateType(table, type); } // The fact we default the name to "[selection]" inconditionally means that any bind marker in a @@ -624,6 +643,7 @@ public Factory newSelectorFactory(TableMetadata cfm, if (type == null) throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)", this); + validateType(cfm, type); } if (selectables.size() == 1 && !type.isTuple()) @@ -742,6 +762,7 @@ public Factory newSelectorFactory(TableMetadata cfm, if (type == null) throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)", this); + validateType(cfm, type); } ListType listType = (ListType) type; @@ -827,6 +848,7 @@ public Factory newSelectorFactory(TableMetadata cfm, if (type == null) throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)", this); + validateType(cfm, type); } // The parser treats empty Maps as Sets so if the type is a MapType we know that the Map is empty @@ -931,6 +953,7 @@ public Factory newSelectorFactory(TableMetadata cfm, if (type == null) throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)", this); + validateType(cfm, type); } if (type.isUDT()) diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java index 7a748e8aaf8b..d7b03d32192d 100644 --- a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java @@ -25,6 +25,7 @@ import org.apache.cassandra.cql3.RoleName; import org.apache.cassandra.exceptions.*; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -47,7 +48,8 @@ public AlterRoleStatement(RoleName name, RoleOptions opts, DCPermissions dcPermi this.dcPermissions = dcPermissions; } - public void validate(ClientState state) throws RequestValidationException + @Override + public void validate(QueryState state) throws RequestValidationException { opts.validate(); @@ -60,7 +62,7 @@ public void validate(ClientState state) throws RequestValidationException throw new InvalidRequestException("ALTER [ROLE|USER] can't be empty"); // validate login here before authorize to avoid leaking user existence to anonymous users. - state.ensureNotAnonymous(); + state.getClientState().ensureNotAnonymous(); if (!DatabaseDescriptor.getRoleManager().isExistingRole(role)) throw new InvalidRequestException(String.format("%s doesn't exist", role.getRoleName())); } diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java index 80bd437779a4..1fafabbc465e 100644 --- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java @@ -18,7 +18,15 @@ package org.apache.cassandra.cql3.statements; import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; @@ -31,17 +39,36 @@ import org.apache.cassandra.audit.AuditLogContext; import org.apache.cassandra.audit.AuditLogEntryType; -import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.*; -import org.apache.cassandra.db.*; +import org.apache.cassandra.cql3.Attributes; +import org.apache.cassandra.cql3.BatchQueryOptions; +import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.ResultSet; +import org.apache.cassandra.cql3.VariableSpecifications; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.IMutation; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.Slices; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.RowIterator; -import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.RequestValidationException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.metrics.BatchMetrics; -import org.apache.cassandra.service.*; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.ClientWarn; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.StorageProxy; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.FBUtilities; @@ -49,7 +76,6 @@ import org.apache.cassandra.utils.Pair; import static java.util.function.Predicate.isEqual; - import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; /** @@ -79,10 +105,6 @@ public enum Type private static final Logger logger = LoggerFactory.getLogger(BatchStatement.class); - private static final String UNLOGGED_BATCH_WARNING = "Unlogged batch covering {} partitions detected " + - "against table{} {}. You should use a logged batch for " + - "atomicity, or asynchronous writes for performance."; - private static final String LOGGED_BATCH_LOW_GCGS_WARNING = "Executing a LOGGED BATCH on table{} {}, configured with a " + "gc_grace_seconds of 0. The gc_grace_seconds is used to TTL " + "batchlog entries, so setting gc_grace_seconds too low on " + @@ -252,8 +274,12 @@ private boolean isLogged() // The batch itself will be validated in either Parsed#prepare() - for regular CQL3 batches, // or in QueryProcessor.processBatch() - for native protocol batches. - public void validate(ClientState state) throws InvalidRequestException + @Override + public void validate(QueryState state) throws InvalidRequestException { + if (isLogged()) + Guardrails.loggedBatchEnabled.ensureEnabled(state); + for (ModificationStatement statement : statements) statement.validate(state); } @@ -264,7 +290,8 @@ public List getStatements() } @VisibleForTesting - public List getMutations(BatchQueryOptions options, + public List getMutations(QueryState state, + BatchQueryOptions options, boolean local, long batchTimestamp, int nowInSeconds, @@ -280,7 +307,7 @@ public List getMutations(BatchQueryOptions options, ModificationStatement stmt = statements.get(i); if (metadata != null && !stmt.metadata.id.equals(metadata.id)) metadata = null; - List stmtPartitionKeys = stmt.buildPartitionKeyNames(options.forStatement(i)); + List stmtPartitionKeys = stmt.buildPartitionKeyNames(options.forStatement(i), state); partitionKeys.add(stmtPartitionKeys); HashMultiset perKeyCountsForTable = partitionCounts.computeIfAbsent(stmt.metadata.id, k -> HashMultiset.create()); for (int stmtIdx = 0, stmtSize = stmtPartitionKeys.size(); stmtIdx < stmtSize; stmtIdx++) @@ -305,7 +332,7 @@ public List getMutations(BatchQueryOptions options, } QueryOptions statementOptions = options.forStatement(i); long timestamp = attrs.getTimestamp(batchTimestamp, statementOptions); - statement.addUpdates(collector, partitionKeys.get(i), statementOptions, local, timestamp, nowInSeconds, queryStartNanoTime); + statement.addUpdates(collector, partitionKeys.get(i), state, statementOptions, local, timestamp, nowInSeconds, queryStartNanoTime); } if (tablesWithZeroGcGs != null) @@ -324,16 +351,15 @@ public List getMutations(BatchQueryOptions options, * * @param mutations - the batch mutations. */ - private static void verifyBatchSize(Collection mutations) throws InvalidRequestException + private static void verifyBatchSize(Collection mutations, QueryState queryState) throws InvalidRequestException { // We only warn for batch spanning multiple mutations (#10876) if (mutations.size() <= 1) return; - long warnThreshold = DatabaseDescriptor.getBatchSizeWarnThreshold(); long size = IMutation.dataSize(mutations); - if (size > warnThreshold) + if (Guardrails.batchSize.triggersOn(size, queryState)) { Set tableNames = new HashSet<>(); for (IMutation mutation : mutations) @@ -342,27 +368,11 @@ private static void verifyBatchSize(Collection mutations) t tableNames.add(update.metadata().toString()); } - long failThreshold = DatabaseDescriptor.getBatchSizeFailThreshold(); - - String format = "Batch for {} is of size {}, exceeding specified threshold of {} by {}.{}"; - if (size > failThreshold) - { - Tracing.trace(format, tableNames, FBUtilities.prettyPrintMemory(size), FBUtilities.prettyPrintMemory(failThreshold), - FBUtilities.prettyPrintMemory(size - failThreshold), " (see batch_size_fail_threshold_in_kb)"); - logger.error(format, tableNames, FBUtilities.prettyPrintMemory(size), FBUtilities.prettyPrintMemory(failThreshold), - FBUtilities.prettyPrintMemory(size - failThreshold), " (see batch_size_fail_threshold_in_kb)"); - throw new InvalidRequestException("Batch too large"); - } - else if (logger.isWarnEnabled()) - { - logger.warn(format, tableNames, FBUtilities.prettyPrintMemory(size), FBUtilities.prettyPrintMemory(warnThreshold), - FBUtilities.prettyPrintMemory(size - warnThreshold), ""); - } - ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[] {tableNames, size, warnThreshold, size - warnThreshold, ""}).getMessage()); + Guardrails.batchSize.guard(size, tableNames.toString(), false, queryState); } } - private void verifyBatchType(Collection mutations) + private void verifyBatchType(Collection mutations, QueryState queryState) { if (!isLogged() && mutations.size() > 1) { @@ -381,13 +391,9 @@ private void verifyBatchType(Collection mutations) // CASSANDRA-11529: log only if we have more than a threshold of keys, this was also suggested in the // original ticket that introduced this warning, CASSANDRA-9282 - if (keySet.size() > DatabaseDescriptor.getUnloggedBatchAcrossPartitionsWarnThreshold()) + if (Guardrails.unloggedBatchAcrossPartitions.triggersOn(keySet.size(), queryState)) { - NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, UNLOGGED_BATCH_WARNING, - keySet.size(), tableNames.size() == 1 ? "" : "s", tableNames); - - ClientWarn.instance.warn(MessageFormatter.arrayFormat(UNLOGGED_BATCH_WARNING, new Object[]{keySet.size(), - tableNames.size() == 1 ? "" : "s", tableNames}).getMessage()); + Guardrails.unloggedBatchAcrossPartitions.guard(keySet.size(), tableNames.toString(), false, queryState); } } } @@ -403,9 +409,18 @@ public ResultMessage execute(QueryState queryState, BatchQueryOptions options, l long timestamp = options.getTimestamp(queryState); int nowInSeconds = options.getNowInSeconds(queryState); - if (options.getConsistency() == null) + ConsistencyLevel cl = options.getConsistency(); + if (cl == null) throw new InvalidRequestException("Invalid empty consistency level"); - if (options.getSerialConsistency() == null) + + for (int i = 0; i < statements.size(); i++ ) + { + ModificationStatement statement = statements.get(i); + statement.validateConsistency(cl, queryState); + statement.validateDiskUsage(queryState, options.forStatement(i)); + } + + if (options.getSerialConsistency(queryState) == null) throw new InvalidRequestException("Invalid empty serial consistency level"); if (hasConditions) @@ -414,18 +429,22 @@ public ResultMessage execute(QueryState queryState, BatchQueryOptions options, l if (updatesVirtualTables) executeInternalWithoutCondition(queryState, options, queryStartNanoTime); else - executeWithoutConditions(getMutations(options, false, timestamp, nowInSeconds, queryStartNanoTime), options.getConsistency(), queryStartNanoTime); + executeWithoutConditions(getMutations(queryState, options, false, timestamp, nowInSeconds, queryStartNanoTime), + queryState, cl, queryStartNanoTime); return new ResultMessage.Void(); } - private void executeWithoutConditions(List mutations, ConsistencyLevel cl, long queryStartNanoTime) throws RequestExecutionException, RequestValidationException + private void executeWithoutConditions(List mutations, + QueryState queryState, + ConsistencyLevel cl, + long queryStartNanoTime) throws RequestExecutionException, RequestValidationException { if (mutations.isEmpty()) return; - verifyBatchSize(mutations); - verifyBatchType(mutations); + verifyBatchSize(mutations, queryState); + verifyBatchType(mutations, queryState); updatePartitionsPerBatchMetrics(mutations.size()); @@ -457,9 +476,9 @@ private ResultMessage executeWithConditions(BatchQueryOptions options, QueryStat tableName, casRequest.key, casRequest, - options.getSerialConsistency(), + options.getSerialConsistency(state), options.getConsistency(), - state.getClientState(), + state, options.getNowInSeconds(state), queryStartNanoTime)) { @@ -486,7 +505,7 @@ private Pair> makeCasRequest(BatchQueryOption ModificationStatement statement = statements.get(i); QueryOptions statementOptions = options.forStatement(i); long timestamp = attrs.getTimestamp(batchTimestamp, statementOptions); - List pks = statement.buildPartitionKeyNames(statementOptions); + List pks = statement.buildPartitionKeyNames(statementOptions, state); if (statement.getRestrictions().keyIsInRelation()) throw new IllegalArgumentException("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)"); if (key == null) @@ -521,7 +540,7 @@ else if (!key.getKey().equals(pks.get(0))) } else { - Clustering clustering = Iterables.getOnlyElement(statement.createClustering(statementOptions)); + Clustering clustering = Iterables.getOnlyElement(statement.createClustering(statementOptions, state)); if (statement.hasConditions()) { statement.addConditions(clustering, casRequest, statementOptions); @@ -559,7 +578,7 @@ private ResultMessage executeInternalWithoutCondition(QueryState queryState, Bat long timestamp = batchOptions.getTimestamp(queryState); int nowInSeconds = batchOptions.getNowInSeconds(queryState); - for (IMutation mutation : getMutations(batchOptions, true, timestamp, nowInSeconds, queryStartNanoTime)) + for (IMutation mutation : getMutations(queryState, batchOptions, true, timestamp, nowInSeconds, queryStartNanoTime)) mutation.apply(); return null; } @@ -576,7 +595,7 @@ private ResultMessage executeInternalWithConditions(BatchQueryOptions options, Q long timestamp = options.getTimestamp(state); int nowInSeconds = options.getNowInSeconds(state); - try (RowIterator result = ModificationStatement.casInternal(request, timestamp, nowInSeconds)) + try (RowIterator result = ModificationStatement.casInternal(request, timestamp, nowInSeconds, state)) { ResultSet resultSet = ModificationStatement.buildCasResultSet(ksName, diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java index 563a63907ff3..d29f7a6c6c5e 100644 --- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java +++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java @@ -34,6 +34,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.service.CASRequest; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.utils.Pair; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -228,13 +229,14 @@ private RegularAndStaticColumns updatedColumns() return builder.build(); } - public PartitionUpdate makeUpdates(FilteredPartition current) throws InvalidRequestException + @Override + public PartitionUpdate makeUpdates(FilteredPartition current, QueryState state) throws InvalidRequestException { PartitionUpdate.Builder updateBuilder = new PartitionUpdate.Builder(metadata, key, updatedColumns(), conditions.size()); for (RowUpdate upd : updates) - upd.applyUpdates(current, updateBuilder); + upd.applyUpdates(current, updateBuilder, state); for (RangeDeletion upd : rangeDeletions) - upd.applyUpdates(current, updateBuilder); + upd.applyUpdates(current, updateBuilder, state); PartitionUpdate partitionUpdate = updateBuilder.build(); IndexRegistry.obtain(metadata).validate(partitionUpdate); @@ -265,12 +267,13 @@ private RowUpdate(Clustering clustering, ModificationStatement stmt, QueryOpt this.nowInSeconds = nowInSeconds; } - void applyUpdates(FilteredPartition current, PartitionUpdate.Builder updateBuilder) + void applyUpdates(FilteredPartition current, PartitionUpdate.Builder updateBuilder, QueryState state) { Map map = stmt.requiresRead() ? Collections.singletonMap(key, current) : null; UpdateParameters params = new UpdateParameters(metadata, updateBuilder.columns(), + state, options, timestamp, nowInSeconds, @@ -297,13 +300,14 @@ private RangeDeletion(Slice slice, ModificationStatement stmt, QueryOptions opti this.nowInSeconds = nowInSeconds; } - void applyUpdates(FilteredPartition current, PartitionUpdate.Builder updateBuilder) + void applyUpdates(FilteredPartition current, PartitionUpdate.Builder updateBuilder, QueryState state) { // No slice statements currently require a read, but this maintains consistency with RowUpdate, and future proofs us Map map = stmt.requiresRead() ? Collections.singletonMap(key, current) : null; UpdateParameters params = new UpdateParameters(metadata, updateBuilder.columns(), + state, options, timestamp, nowInSeconds, diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java index 574d661d0e28..a15f0b127f10 100644 --- a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java @@ -24,6 +24,7 @@ import org.apache.cassandra.cql3.RoleName; import org.apache.cassandra.exceptions.*; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -53,7 +54,8 @@ public void authorize(ClientState state) throws UnauthorizedException } } - public void validate(ClientState state) throws RequestValidationException + @Override + public void validate(QueryState state) throws RequestValidationException { opts.validate(); @@ -66,7 +68,7 @@ public void validate(ClientState state) throws RequestValidationException throw new InvalidRequestException("Role name can't be an empty string"); // validate login here before authorize to avoid leaking role existence to anonymous users. - state.ensureNotAnonymous(); + state.getClientState().ensureNotAnonymous(); if (!ifNotExists && DatabaseDescriptor.getRoleManager().isExistingRole(role)) throw new InvalidRequestException(String.format("%s already exists", role.getRoleName())); diff --git a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java index 16671dd38d9a..db78df50958d 100644 --- a/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/DescribeStatement.java @@ -37,6 +37,7 @@ import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.OperationExecutionException; import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.exceptions.RequestValidationException; import org.apache.cassandra.io.util.DataInputBuffer; @@ -114,7 +115,7 @@ public final void authorize(ClientState state) } @Override - public final void validate(ClientState state) + public final void validate(QueryState state) { } @@ -157,14 +158,17 @@ public ResultMessage executeLocally(QueryState state, QueryOptions options) // long offset = getOffset(pagingState, schemaVersion); - int pageSize = options.getPageSize(); + PageSize pageSize = options.getPageSize(); + + if (pageSize.isDefined() && pageSize.getUnit() != PageSize.PageUnit.ROWS) + throw new OperationExecutionException("Paging in bytes is not supported for describe statement. Please specify the page size in rows."); Stream stream = describe(state.getClientState(), keyspaces); if (offset > 0L) stream = stream.skip(offset); - if (pageSize > 0) - stream = stream.limit(pageSize); + if (pageSize.isDefined()) + stream = stream.limit(pageSize.getSize()); List> rows = stream.map(e -> toRow(e, includeInternalDetails)) .collect(Collectors.toList()); @@ -172,9 +176,9 @@ public ResultMessage executeLocally(QueryState state, QueryOptions options) ResultSet.ResultMetadata resultMetadata = new ResultSet.ResultMetadata(metadata(state.getClientState())); ResultSet result = new ResultSet(resultMetadata, rows); - if (pageSize > 0 && rows.size() == pageSize) + if (pageSize.isDefined() && rows.size() == pageSize.getSize()) { - result.metadata.setHasMorePages(getPagingState(offset + pageSize, schemaVersion)); + result.metadata.setHasMorePages(getPagingState(offset + pageSize.getSize(), schemaVersion)); } return new ResultMessage.Rows(result); diff --git a/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java index 058ab01c04ba..e7c81c2671b4 100644 --- a/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java @@ -24,6 +24,7 @@ import org.apache.cassandra.cql3.RoleName; import org.apache.cassandra.exceptions.*; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -51,15 +52,16 @@ public void authorize(ClientState state) throws UnauthorizedException throw new UnauthorizedException("Only superusers can drop a role with superuser status"); } - public void validate(ClientState state) throws RequestValidationException + @Override + public void validate(QueryState state) throws RequestValidationException { // validate login here before authorize to avoid leaking user existence to anonymous users. - state.ensureNotAnonymous(); + state.getClientState().ensureNotAnonymous(); if (!ifExists && !DatabaseDescriptor.getRoleManager().isExistingRole(role)) throw new InvalidRequestException(String.format("%s doesn't exist", role.getRoleName())); - AuthenticatedUser user = state.getUser(); + AuthenticatedUser user = state.getClientState().getUser(); if (user != null && user.getName().equals(role.getRoleName())) throw new InvalidRequestException("Cannot DROP primary role for current login"); } diff --git a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java index 4b5aa601e2ab..3a39fb0e7da9 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java @@ -30,6 +30,7 @@ import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.exceptions.RequestValidationException; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -64,14 +65,15 @@ public ListPermissionsStatement(Set permissions, IResource resource, this.grantee = grantee.hasName()? RoleResource.role(grantee.getName()) : null; } - public void validate(ClientState state) throws RequestValidationException + @Override + public void validate(QueryState state) throws RequestValidationException { // a check to ensure the existence of the user isn't being leaked by user existence check. - state.ensureNotAnonymous(); + state.getClientState().ensureNotAnonymous(); if (resource != null) { - resource = maybeCorrectResource(resource, state); + resource = maybeCorrectResource(resource, state.getClientState()); if (!resource.exists()) throw new InvalidRequestException(String.format("%s doesn't exist", resource)); } diff --git a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java index 8a75f8a6c36a..8bebe3425062 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java @@ -35,6 +35,7 @@ import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.exceptions.*; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -67,9 +68,10 @@ public ListRolesStatement(RoleName grantee, boolean recursive) this.recursive = recursive; } - public void validate(ClientState state) throws UnauthorizedException, InvalidRequestException + @Override + public void validate(QueryState state) throws UnauthorizedException, InvalidRequestException { - state.ensureNotAnonymous(); + state.getClientState().ensureNotAnonymous(); if ((grantee != null) && !DatabaseDescriptor.getRoleManager().isExistingRole(grantee)) throw new InvalidRequestException(String.format("%s doesn't exist", grantee)); diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java index 087f3b0e7c4b..4ac3a53565ae 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java @@ -49,9 +49,18 @@ import org.apache.cassandra.db.rows.RowIterator; import org.apache.cassandra.db.view.View; import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.guardrails.Guardrails; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.ViewMetadata; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.disk.usage.DiskUsageBroadcaster; import org.apache.cassandra.service.paxos.Commit; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.triggers.TriggerExecutor; @@ -244,23 +253,14 @@ public void authorize(ClientState state) throws InvalidRequestException, Unautho if (hasConditions()) state.ensureTablePermission(metadata, Permission.SELECT); - // MV updates need to get the current state from the table, and might update the views - // Require Permission.SELECT on the base table, and Permission.MODIFY on the views - Iterator views = View.findAll(keyspace(), columnFamily()).iterator(); - if (views.hasNext()) - { - state.ensureTablePermission(metadata, Permission.SELECT); - do - { - state.ensureTablePermission(views.next().metadata, Permission.MODIFY); - } while (views.hasNext()); - } + // Modification on base table with MV should skip SELECT access control to base table and WRITE access control to view table. for (Function function : getFunctions()) state.ensurePermission(Permission.EXECUTE, function); } - public void validate(ClientState state) throws InvalidRequestException + @Override + public void validate(QueryState state) throws InvalidRequestException { checkFalse(hasConditions() && attrs.isTimestampSet(), "Cannot provide custom timestamp for conditional updates"); checkFalse(isCounter() && attrs.isTimestampSet(), "Cannot provide custom timestamp for counter updates"); @@ -268,6 +268,10 @@ public void validate(ClientState state) throws InvalidRequestException checkFalse(isView(), "Cannot directly modify a materialized view"); checkFalse(isVirtual() && attrs.isTimeToLiveSet(), "Expiring columns are not supported by virtual tables"); checkFalse(isVirtual() && hasConditions(), "Conditional updates are not supported by virtual tables"); + + // there are system queries with USING TIMESTAMP, e.g. SchemaKeyspace#saveSystemKeyspacesSchema + if (SchemaConstants.isUserKeyspace(metadata.keyspace) && attrs.isTimestampSet()) + Guardrails.userTimestampsEnabled.ensureEnabled(state); } public RegularAndStaticColumns updatedColumns() @@ -325,23 +329,23 @@ public boolean hasIfExistCondition() return conditions.isIfExists(); } - public List buildPartitionKeyNames(QueryOptions options) + public List buildPartitionKeyNames(QueryOptions options, QueryState queryState) throws InvalidRequestException { - List partitionKeys = restrictions.getPartitionKeys(options); + List partitionKeys = restrictions.getPartitionKeys(options, queryState); for (ByteBuffer key : partitionKeys) QueryProcessor.validateKey(key); return partitionKeys; } - public NavigableSet> createClustering(QueryOptions options) + public NavigableSet> createClustering(QueryOptions options, QueryState queryState) throws InvalidRequestException { if (appliesOnlyToStaticColumns() && !restrictions.hasClusteringColumnsRestrictions()) return FBUtilities.singleton(CBuilder.STATIC_BUILDER.build(), metadata().comparator); - return restrictions.getClusteringColumns(options); + return restrictions.getClusteringColumns(options, queryState); } /** @@ -353,6 +357,19 @@ private boolean appliesOnlyToStaticColumns() return appliesOnlyToStaticColumns(operations, conditions); } + public void validateDiskUsage(QueryState state, QueryOptions options) + { + // reject writes if any replica exceeds disk usage failure limit or warn if exceeds warn limit + if (Guardrails.replicaDiskUsage.enabled(state) && DiskUsageBroadcaster.instance.hasStuffedOrFullNode()) + { + for (ByteBuffer keyValue : buildPartitionKeyNames(options, state)) + { + for (InetAddressAndPort replica : StorageService.instance.getNaturalReplicasForToken(keyspace(), keyValue).endpointList()) + Guardrails.replicaDiskUsage.guard(replica, state); + } + } + } + /** * Checks that the specified operations and conditions only apply to static columns. * @return true if the specified operations and conditions only apply to static columns, @@ -460,23 +477,29 @@ private ResultMessage executeWithoutCondition(QueryState queryState, QueryOption return executeInternalWithoutCondition(queryState, options, queryStartNanoTime); ConsistencyLevel cl = options.getConsistency(); - if (isCounter()) - cl.validateCounterForWrite(metadata()); - else - cl.validateForWrite(); + validateConsistency(cl, queryState); + validateDiskUsage(queryState, options); List mutations = - getMutations(options, - false, - options.getTimestamp(queryState), - options.getNowInSeconds(queryState), - queryStartNanoTime); + getMutations(queryState, + options, + false, + options.getTimestamp(queryState), + options.getNowInSeconds(queryState), queryStartNanoTime); if (!mutations.isEmpty()) StorageProxy.mutateWithTriggers(mutations, cl, false, queryStartNanoTime); return null; } + public void validateConsistency(ConsistencyLevel cl, QueryState queryState) + { + if (isCounter()) + cl.validateCounterForWrite(metadata(), queryState); + else + cl.validateForWrite(metadata.keyspace, queryState); + } + private ResultMessage executeWithCondition(QueryState queryState, QueryOptions options, long queryStartNanoTime) { CQL3CasRequest request = makeCasRequest(queryState, options); @@ -485,9 +508,9 @@ private ResultMessage executeWithCondition(QueryState queryState, QueryOptions o columnFamily(), request.key, request, - options.getSerialConsistency(), + options.getSerialConsistency(queryState), options.getConsistency(), - queryState.getClientState(), + queryState, options.getNowInSeconds(queryState), queryStartNanoTime)) { @@ -497,7 +520,7 @@ private ResultMessage executeWithCondition(QueryState queryState, QueryOptions o private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions options) { - List keys = buildPartitionKeyNames(options); + List keys = buildPartitionKeyNames(options, queryState); // We don't support IN for CAS operation so far checkFalse(restrictions.keyIsInRelation(), "IN on the partition key is not supported with conditional %s", @@ -511,7 +534,7 @@ private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions option "IN on the clustering key columns is not supported with conditional %s", type.isUpdate()? "updates" : "deletions"); - Clustering clustering = Iterables.getOnlyElement(createClustering(options)); + Clustering clustering = Iterables.getOnlyElement(createClustering(options, queryState)); CQL3CasRequest request = new CQL3CasRequest(metadata(), key, conditionColumns(), updatesRegularRows(), updatesStaticRow()); addConditions(clustering, request, options); @@ -629,7 +652,7 @@ public ResultMessage executeInternalWithoutCondition(QueryState queryState, Quer { long timestamp = options.getTimestamp(queryState); int nowInSeconds = options.getNowInSeconds(queryState); - for (IMutation mutation : getMutations(options, true, timestamp, nowInSeconds, queryStartNanoTime)) + for (IMutation mutation : getMutations(queryState, options, true, timestamp, nowInSeconds, queryStartNanoTime)) mutation.apply(); return null; } @@ -638,13 +661,13 @@ public ResultMessage executeInternalWithCondition(QueryState state, QueryOptions { CQL3CasRequest request = makeCasRequest(state, options); - try (RowIterator result = casInternal(request, options.getTimestamp(state), options.getNowInSeconds(state))) + try (RowIterator result = casInternal(request, options.getTimestamp(state), options.getNowInSeconds(state), state)) { return new ResultMessage.Rows(buildCasResultSet(result, state, options)); } } - static RowIterator casInternal(CQL3CasRequest request, long timestamp, int nowInSeconds) + static RowIterator casInternal(CQL3CasRequest request, long timestamp, int nowInSeconds, QueryState state) { UUID ballot = UUIDGen.getTimeUUIDFromMicros(timestamp); @@ -659,7 +682,7 @@ static RowIterator casInternal(CQL3CasRequest request, long timestamp, int nowIn if (!request.appliesTo(current)) return current.rowIterator(); - PartitionUpdate updates = request.makeUpdates(current); + PartitionUpdate updates = request.makeUpdates(current, state); updates = TriggerExecutor.instance.execute(updates); Commit proposal = Commit.newProposal(ballot, updates); @@ -670,27 +693,31 @@ static RowIterator casInternal(CQL3CasRequest request, long timestamp, int nowIn /** * Convert statement into a list of mutations to apply on the server * + * + * @param state The query state * @param options value for prepared statement markers * @param local if true, any requests (for collections) performed by getMutation should be done locally only. * @param timestamp the current timestamp in microseconds to use if no timestamp is user provided. * * @return list of the mutations */ - private List getMutations(QueryOptions options, - boolean local, - long timestamp, - int nowInSeconds, - long queryStartNanoTime) - { - List keys = buildPartitionKeyNames(options); + private List getMutations(QueryState state, + QueryOptions options, + boolean local, + long timestamp, + int nowInSeconds, + long queryStartNanoTime) + { + List keys = buildPartitionKeyNames(options, state); HashMultiset perPartitionKeyCounts = HashMultiset.create(keys); SingleTableUpdatesCollector collector = new SingleTableUpdatesCollector(metadata, updatedColumns, perPartitionKeyCounts); - addUpdates(collector, keys, options, local, timestamp, nowInSeconds, queryStartNanoTime); + addUpdates(collector, keys, state, options, local, timestamp, nowInSeconds, queryStartNanoTime); return collector.toMutations(); } final void addUpdates(UpdatesCollector collector, List keys, + QueryState state, QueryOptions options, boolean local, long timestamp, @@ -707,6 +734,7 @@ final void addUpdates(UpdatesCollector collector, UpdateParameters params = makeUpdateParameters(keys, new ClusteringIndexSliceFilter(slices, false), + state, options, DataLimits.NONE, local, @@ -726,13 +754,13 @@ final void addUpdates(UpdatesCollector collector, } else { - NavigableSet> clusterings = createClustering(options); + NavigableSet> clusterings = createClustering(options, state); // If some of the restrictions were unspecified (e.g. empty IN restrictions) we do not need to do anything. if (restrictions.hasClusteringColumnsRestrictions() && clusterings.isEmpty()) return; - UpdateParameters params = makeUpdateParameters(keys, clusterings, options, local, timestamp, nowInSeconds, queryStartNanoTime); + UpdateParameters params = makeUpdateParameters(keys, clusterings, state, options, local, timestamp, nowInSeconds, queryStartNanoTime); for (ByteBuffer key : keys) { @@ -779,6 +807,7 @@ Slices createSlices(QueryOptions options) private UpdateParameters makeUpdateParameters(Collection keys, NavigableSet> clusterings, + QueryState state, QueryOptions options, boolean local, long timestamp, @@ -788,6 +817,7 @@ private UpdateParameters makeUpdateParameters(Collection keys, if (clusterings.contains(Clustering.STATIC_CLUSTERING)) return makeUpdateParameters(keys, new ClusteringIndexSliceFilter(Slices.ALL, false), + state, options, DataLimits.cqlLimits(1), local, @@ -797,6 +827,7 @@ private UpdateParameters makeUpdateParameters(Collection keys, return makeUpdateParameters(keys, new ClusteringIndexNamesFilter(clusterings, false), + state, options, DataLimits.NONE, local, @@ -807,6 +838,7 @@ private UpdateParameters makeUpdateParameters(Collection keys, private UpdateParameters makeUpdateParameters(Collection keys, ClusteringIndexFilter filter, + QueryState state, QueryOptions options, DataLimits limits, boolean local, @@ -826,6 +858,7 @@ private UpdateParameters makeUpdateParameters(Collection keys, return new UpdateParameters(metadata(), updatedColumns(), + state, options, getTimestamp(timestamp, options), nowInSeconds, @@ -975,7 +1008,7 @@ protected StatementRestrictions newRestrictions(TableMetadata metadata, throw new InvalidRequestException(CUSTOM_EXPRESSIONS_NOT_ALLOWED); boolean applyOnlyToStaticColumns = appliesOnlyToStaticColumns(operations, conditions); - return new StatementRestrictions(type, metadata, where, boundNames, applyOnlyToStaticColumns, false, false); + return StatementRestrictions.create(type, metadata, where, boundNames, applyOnlyToStaticColumns, false, false); } public List> getConditions() diff --git a/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java index aa7e85ba7307..e29fa56f99e3 100644 --- a/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java @@ -27,6 +27,8 @@ import org.apache.cassandra.exceptions.RequestValidationException; import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; + import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -43,17 +45,18 @@ protected PermissionsManagementStatement(Set permissions, IResource this.grantee = RoleResource.role(grantee.getName()); } - public void validate(ClientState state) throws RequestValidationException + @Override + public void validate(QueryState state) throws RequestValidationException { // validate login here before authorize to avoid leaking user existence to anonymous users. - state.ensureNotAnonymous(); + state.getClientState().ensureNotAnonymous(); if (!DatabaseDescriptor.getRoleManager().isExistingRole(grantee)) throw new InvalidRequestException(String.format("Role %s doesn't exist", grantee.getRoleName())); // if a keyspace is omitted when GRANT/REVOKE ON TABLE
, we need to correct the resource. // called both here and in authorize(), as in some cases we do not call the latter. - resource = maybeCorrectResource(resource, state); + resource = maybeCorrectResource(resource, state.getClientState()); // altering permissions on builtin functions is not supported if (resource instanceof FunctionResource diff --git a/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java b/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java index 590910f47376..0123aec79145 100644 --- a/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java +++ b/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java @@ -69,7 +69,7 @@ protected String getSimple(String name) throws SyntaxException return (String)val; } - protected Map getMap(String name) throws SyntaxException + public Map getMap(String name) throws SyntaxException { Object val = properties.get(name); if (val == null) @@ -143,4 +143,17 @@ public static Integer toInt(String key, String value, Integer defaultValue) thro } } } + + /** + * Returns the name of all the properties that are updated by this object. + */ + public Set updatedProperties() + { + return properties.keySet(); + } + + public void removeProperty(String name) + { + properties.remove(name); + } } diff --git a/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java index a5274dd73834..1d8560f3bb44 100644 --- a/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java @@ -25,6 +25,8 @@ import org.apache.cassandra.exceptions.RequestValidationException; import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; + import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -44,9 +46,10 @@ public void authorize(ClientState state) throws UnauthorizedException super.checkPermission(state, Permission.AUTHORIZE, role); } - public void validate(ClientState state) throws RequestValidationException + @Override + public void validate(QueryState state) throws RequestValidationException { - state.ensureNotAnonymous(); + state.getClientState().ensureNotAnonymous(); if (!DatabaseDescriptor.getRoleManager().isExistingRole(role)) throw new InvalidRequestException(String.format("%s doesn't exist", role.getRoleName())); diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 774bd689799d..abe0882503b3 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -29,8 +29,11 @@ import org.apache.cassandra.audit.AuditLogContext; import org.apache.cassandra.audit.AuditLogEntryType; import org.apache.cassandra.auth.Permission; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.cql3.*; @@ -67,6 +70,8 @@ import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; + import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -74,6 +79,7 @@ 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.db.filter.DataLimits.NO_LIMIT; import static org.apache.cassandra.utils.ByteBufferUtil.UNSET_BYTE_BUFFER; /** @@ -89,8 +95,6 @@ public class SelectStatement implements CQLStatement { private static final Logger logger = LoggerFactory.getLogger(SelectStatement.class); - public static final int DEFAULT_PAGE_SIZE = 10000; - public final VariableSpecifications bindVariables; public final TableMetadata table; public final Parameters parameters; @@ -192,7 +196,7 @@ static SelectStatement forSelection(TableMetadata table, Selection selection) VariableSpecifications.empty(), defaultParameters, selection, - StatementRestrictions.empty(StatementType.SELECT, table), + StatementRestrictions.empty(table), false, null, null, @@ -222,32 +226,61 @@ public void authorize(ClientState state) throws InvalidRequestException, Unautho state.ensurePermission(Permission.EXECUTE, function); } - public void validate(ClientState state) throws InvalidRequestException + @Override + public void validate(QueryState state) throws InvalidRequestException { // Nothing to do, all validation has been done by RawStatement.prepare() } - public ResultMessage.Rows execute(QueryState state, QueryOptions options, long queryStartNanoTime) + private void validateQueryOptions(QueryState queryState, QueryOptions options) + { + if (SchemaConstants.isUserKeyspace(table.keyspace)) + Guardrails.disallowedWriteConsistencies.ensureAllowed(options.getConsistency(), queryState); + + PageSize pageSize = options.getPageSize(); + if (pageSize != null && options.getPageSize().isDefined() && pageSize.getUnit() == PageSize.PageUnit.BYTES) + { + Guardrails.pageSize.guard(pageSize.bytes(), "in bytes", false, queryState); + } + } + + /** + * Returns whether the paging can be skipped based on the user limits and the page size - that is, if the user limit + * is provided and is lower than the page size, it means that we will only return at most one page and thus paging + * is unnecessary in this case. That applies to the page size defined in rows - if the page size is defined in bytes + * we cannot say anything about the relation beteween the user rows limit and the page size. + */ + private boolean canSkipPaging(DataLimits userLimits, PageSize pageSize) + { + return !pageSize.isDefined() || + pageSize.getUnit() == PageSize.PageUnit.ROWS && !pageSize.isCompleted(userLimits.count(), PageSize.PageUnit.ROWS); + } + + public ResultMessage.Rows execute(QueryState queryState, QueryOptions options, long queryStartNanoTime) { ConsistencyLevel cl = options.getConsistency(); checkNotNull(cl, "Invalid empty consistency level"); cl.validateForRead(); + validateQueryOptions(queryState, options); - int nowInSec = options.getNowInSeconds(state); + int nowInSec = options.getNowInSeconds(queryState); int userLimit = getLimit(options); int userPerPartitionLimit = getPerPartitionLimit(options); - int pageSize = options.getPageSize(); + PageSize pageSize = options.getPageSize(); Selectors selectors = selection.newSelectors(options); - ReadQuery query = getQuery(options, selectors.getColumnFilter(), nowInSec, userLimit, userPerPartitionLimit, pageSize); + ReadQuery query = getQuery(queryState, options, selectors.getColumnFilter(), nowInSec, userLimit, userPerPartitionLimit, pageSize); + + if (query.limits().isGroupByLimit() && pageSize != null && pageSize.isDefined() && pageSize.getUnit() == PageSize.PageUnit.BYTES) + throw new InvalidRequestException("Paging in bytes cannot be specified for aggregation queries"); - if (aggregationSpec == null && (pageSize <= 0 || (query.limits().count() <= pageSize))) - return execute(query, options, state, selectors, nowInSec, userLimit, queryStartNanoTime); + if (aggregationSpec == null && canSkipPaging(query.limits(), pageSize)) + return execute(query, options, queryState, selectors, nowInSec, userLimit, queryStartNanoTime); QueryPager pager = getPager(query, options); - return execute(Pager.forDistributedQuery(pager, cl, state.getClientState()), + return execute(Pager.forDistributedQuery(pager, cl, queryState), options, selectors, pageSize, @@ -256,10 +289,11 @@ public ResultMessage.Rows execute(QueryState state, QueryOptions options, long q queryStartNanoTime); } - public ReadQuery getQuery(QueryOptions options, int nowInSec) throws RequestValidationException + public ReadQuery getQuery(QueryState state, QueryOptions options, int nowInSec) throws RequestValidationException { Selectors selectors = selection.newSelectors(options); - return getQuery(options, + return getQuery(state, + options, selectors.getColumnFilter(), nowInSec, getLimit(options), @@ -267,31 +301,32 @@ public ReadQuery getQuery(QueryOptions options, int nowInSec) throws RequestVali options.getPageSize()); } - public ReadQuery getQuery(QueryOptions options, + public ReadQuery getQuery(QueryState queryState, + QueryOptions options, ColumnFilter columnFilter, int nowInSec, int userLimit, int perPartitionLimit, - int pageSize) + PageSize pageSize) { - boolean isPartitionRangeQuery = restrictions.isKeyRange() || restrictions.usesSecondaryIndexing(); + boolean isPartitionRangeQuery = restrictions.isKeyRange() || restrictions.usesSecondaryIndexing() || restrictions.isDisjunction(); - DataLimits limit = getDataLimits(userLimit, perPartitionLimit, pageSize); + DataLimits limit = getDataLimits(queryState, userLimit, perPartitionLimit); if (isPartitionRangeQuery) - return getRangeCommand(options, columnFilter, limit, nowInSec); + return getRangeCommand(options, columnFilter, limit, nowInSec, queryState); - return getSliceCommands(options, columnFilter, limit, nowInSec); + return getSliceCommands(queryState, options, columnFilter, limit, nowInSec); } private ResultMessage.Rows execute(ReadQuery query, QueryOptions options, - QueryState state, + QueryState queryState, Selectors selectors, int nowInSec, int userLimit, long queryStartNanoTime) throws RequestValidationException, RequestExecutionException { - try (PartitionIterator data = query.execute(options.getConsistency(), state.getClientState(), queryStartNanoTime)) + try (PartitionIterator data = query.execute(options.getConsistency(), queryState, queryStartNanoTime)) { return processResults(data, options, selectors, nowInSec, userLimit); } @@ -318,9 +353,9 @@ public static Pager forInternalQuery(QueryPager pager, ReadExecutionController e return new InternalPager(pager, executionController); } - public static Pager forDistributedQuery(QueryPager pager, ConsistencyLevel consistency, ClientState clientState) + public static Pager forDistributedQuery(QueryPager pager, ConsistencyLevel consistency, QueryState queryState) { - return new NormalPager(pager, consistency, clientState); + return new NormalPager(pager, consistency, queryState); } public boolean isExhausted() @@ -333,23 +368,23 @@ public PagingState state() return pager.state(); } - public abstract PartitionIterator fetchPage(int pageSize, long queryStartNanoTime); + public abstract PartitionIterator fetchPage(PageSize pageSize, long queryStartNanoTime); public static class NormalPager extends Pager { private final ConsistencyLevel consistency; - private final ClientState clientState; + private final QueryState queryState; - private NormalPager(QueryPager pager, ConsistencyLevel consistency, ClientState clientState) + private NormalPager(QueryPager pager, ConsistencyLevel consistency, QueryState queryState) { super(pager); this.consistency = consistency; - this.clientState = clientState; + this.queryState = queryState; } - public PartitionIterator fetchPage(int pageSize, long queryStartNanoTime) + public PartitionIterator fetchPage(PageSize pageSize, long queryStartNanoTime) { - return pager.fetchPage(pageSize, consistency, clientState, queryStartNanoTime); + return pager.fetchPage(pageSize, consistency, queryState, queryStartNanoTime); } } @@ -363,7 +398,7 @@ private InternalPager(QueryPager pager, ReadExecutionController executionControl this.executionController = executionController; } - public PartitionIterator fetchPage(int pageSize, long queryStartNanoTime) + public PartitionIterator fetchPage(PageSize pageSize, long queryStartNanoTime) { return pager.fetchPageInternal(pageSize, executionController); } @@ -373,7 +408,7 @@ public PartitionIterator fetchPage(int pageSize, long queryStartNanoTime) private ResultMessage.Rows execute(Pager pager, QueryOptions options, Selectors selectors, - int pageSize, + PageSize pageSize, int nowInSec, int userLimit, long queryStartNanoTime) throws RequestValidationException, RequestExecutionException @@ -392,7 +427,7 @@ else if (restrictions.keyIsInRelation()) // We can't properly do post-query ordering if we page (see #6722) // For GROUP BY or aggregation queries we always page internally even if the user has turned paging off - checkFalse(pageSize > 0 && needsPostQueryOrdering(), + checkFalse(pageSize.isDefined() && needsPostQueryOrdering(), "Cannot page queries with both ORDER BY and a IN restriction on the partition key;" + " you must either remove the ORDER BY or the IN and sort client side, or disable paging for this query"); @@ -435,14 +470,14 @@ public ResultMessage.Rows executeInternal(QueryState state, QueryOptions options { int userLimit = getLimit(options); int userPerPartitionLimit = getPerPartitionLimit(options); - int pageSize = options.getPageSize(); + PageSize pageSize = options.getPageSize(); Selectors selectors = selection.newSelectors(options); - ReadQuery query = getQuery(options, selectors.getColumnFilter(), nowInSec, userLimit, userPerPartitionLimit, pageSize); + ReadQuery query = getQuery(state, options, selectors.getColumnFilter(), nowInSec, userLimit, userPerPartitionLimit, pageSize); try (ReadExecutionController executionController = query.executionController()) { - if (aggregationSpec == null && (pageSize <= 0 || (query.limits().count() <= pageSize))) + if (aggregationSpec == null && canSkipPaging(query.limits(), pageSize)) { try (PartitionIterator data = query.executeInternal(executionController)) { @@ -462,14 +497,15 @@ public ResultMessage.Rows executeInternal(QueryState state, QueryOptions options } } - private QueryPager getPager(ReadQuery query, QueryOptions options) + @VisibleForTesting + public QueryPager getPager(ReadQuery query, QueryOptions options) { QueryPager pager = query.getPager(options.getPagingState(), options.getProtocolVersion()); if (aggregationSpec == null || query.isEmpty()) return pager; - return new AggregationQueryPager(pager, query.limits()); + return new AggregationQueryPager(pager, DatabaseDescriptor.getAggregationSubPageSize(), query.limits()); } public ResultSet process(PartitionIterator partitions, int nowInSec) throws InvalidRequestException @@ -505,13 +541,15 @@ public StatementRestrictions getRestrictions() return restrictions; } - private ReadQuery getSliceCommands(QueryOptions options, ColumnFilter columnFilter, DataLimits limit, int nowInSec) + private ReadQuery getSliceCommands(QueryState queryState, QueryOptions options, ColumnFilter columnFilter, DataLimits limit, int nowInSec) { - Collection keys = restrictions.getPartitionKeys(options); + Collection keys = restrictions.getPartitionKeys(options, queryState); if (keys.isEmpty()) return ReadQuery.empty(table); - ClusteringIndexFilter filter = makeClusteringIndexFilter(options, columnFilter); + Guardrails.partitionKeysInSelectQuery.guard(keys.size(), "Select query", false, queryState); + + ClusteringIndexFilter filter = makeClusteringIndexFilter(options, columnFilter, queryState); if (filter == null || filter.isEmpty(table.comparator)) return ReadQuery.empty(table); @@ -537,9 +575,10 @@ private ReadQuery getSliceCommands(QueryOptions options, ColumnFilter columnFilt */ public Slices clusteringIndexFilterAsSlices() { + QueryState state = QueryState.forInternalCalls(); QueryOptions options = QueryOptions.forInternalCalls(Collections.emptyList()); ColumnFilter columnFilter = selection.newSelectors(options).getColumnFilter(); - ClusteringIndexFilter filter = makeClusteringIndexFilter(options, columnFilter); + ClusteringIndexFilter filter = makeClusteringIndexFilter(options, columnFilter, state); if (filter instanceof ClusteringIndexSliceFilter) return ((ClusteringIndexSliceFilter)filter).requestedSlices(); @@ -555,9 +594,10 @@ public Slices clusteringIndexFilterAsSlices() */ public SinglePartitionReadCommand internalReadForView(DecoratedKey key, int nowInSec) { + QueryState state = QueryState.forInternalCalls(); QueryOptions options = QueryOptions.forInternalCalls(Collections.emptyList()); ColumnFilter columnFilter = selection.newSelectors(options).getColumnFilter(); - ClusteringIndexFilter filter = makeClusteringIndexFilter(options, columnFilter); + ClusteringIndexFilter filter = makeClusteringIndexFilter(options, columnFilter, state); RowFilter rowFilter = getRowFilter(options); return SinglePartitionReadCommand.create(table, nowInSec, columnFilter, rowFilter, DataLimits.NONE, key, filter); } @@ -570,9 +610,9 @@ public RowFilter rowFilterForInternalCalls() return getRowFilter(QueryOptions.forInternalCalls(Collections.emptyList())); } - private ReadQuery getRangeCommand(QueryOptions options, ColumnFilter columnFilter, DataLimits limit, int nowInSec) + private ReadQuery getRangeCommand(QueryOptions options, ColumnFilter columnFilter, DataLimits limit, int nowInSec, QueryState queryState) { - ClusteringIndexFilter clusteringIndexFilter = makeClusteringIndexFilter(options, columnFilter); + ClusteringIndexFilter clusteringIndexFilter = makeClusteringIndexFilter(options, columnFilter, queryState); if (clusteringIndexFilter == null) return ReadQuery.empty(table); @@ -593,7 +633,7 @@ private ReadQuery getRangeCommand(QueryOptions options, ColumnFilter columnFilte return command; } - private ClusteringIndexFilter makeClusteringIndexFilter(QueryOptions options, ColumnFilter columnFilter) + private ClusteringIndexFilter makeClusteringIndexFilter(QueryOptions options, ColumnFilter columnFilter, QueryState queryState) { if (parameters.isDistinct) { @@ -607,6 +647,11 @@ private ClusteringIndexFilter makeClusteringIndexFilter(QueryOptions options, Co return new ClusteringIndexSliceFilter(Slices.ALL, false); } + if (restrictions.isDisjunction()) + { + return new ClusteringIndexSliceFilter(Slices.ALL, false); + } + if (restrictions.isColumnRange()) { Slices slices = makeSlices(options); @@ -616,7 +661,7 @@ private ClusteringIndexFilter makeClusteringIndexFilter(QueryOptions options, Co return new ClusteringIndexSliceFilter(slices, isReversed); } - NavigableSet> clusterings = getRequestedRows(options); + NavigableSet> clusterings = getRequestedRows(options, queryState); // We can have no clusterings if either we're only selecting the static columns, or if we have // a 'IN ()' for clusterings. In that case, we still want to query if some static columns are // queried. But we're fine otherwise. @@ -662,10 +707,10 @@ public Slices makeSlices(QueryOptions options) return builder.build(); } - private DataLimits getDataLimits(int userLimit, int perPartitionLimit, int pageSize) + private DataLimits getDataLimits(QueryState queryState, int userLimit, int perPartitionLimit) { - int cqlRowLimit = DataLimits.NO_LIMIT; - int cqlPerPartitionLimit = DataLimits.NO_LIMIT; + int cqlRowLimit = NO_LIMIT; + int cqlPerPartitionLimit = NO_LIMIT; // If we do post ordering we need to get all the results sorted before we can trim them. if (aggregationSpec != AggregationSpecification.AGGREGATE_EVERYTHING) @@ -675,28 +720,39 @@ private DataLimits getDataLimits(int userLimit, int perPartitionLimit, int pageS cqlPerPartitionLimit = perPartitionLimit; } - // Group by and aggregation queries will always be paged internally to avoid OOM. - // If the user provided a pageSize we'll use that to page internally (because why not), otherwise we use our default - if (pageSize <= 0) - pageSize = DEFAULT_PAGE_SIZE; + DataLimits limits = null; // Aggregation queries work fine on top of the group by paging but to maintain // backward compatibility we need to use the old way. if (aggregationSpec != null && aggregationSpec != AggregationSpecification.AGGREGATE_EVERYTHING) { if (parameters.isDistinct) - return DataLimits.distinctLimits(cqlRowLimit); - - return DataLimits.groupByLimits(cqlRowLimit, - cqlPerPartitionLimit, - pageSize, - aggregationSpec); + limits = DataLimits.distinctLimits(cqlRowLimit); + else + limits = DataLimits.groupByLimits(cqlRowLimit, + cqlPerPartitionLimit, + NO_LIMIT, + NO_LIMIT, + aggregationSpec); + } + else + { + if (parameters.isDistinct) + limits = cqlRowLimit == NO_LIMIT ? DataLimits.DISTINCT_NONE : DataLimits.distinctLimits(cqlRowLimit); + else + limits = DataLimits.cqlLimits(cqlRowLimit, cqlPerPartitionLimit); } - if (parameters.isDistinct) - return cqlRowLimit == DataLimits.NO_LIMIT ? DataLimits.DISTINCT_NONE : DataLimits.distinctLimits(cqlRowLimit); + if (!limits.isGroupByLimit() && Guardrails.pageSize.enabled(queryState)) + { + int bytesLimit = DatabaseDescriptor.getGuardrailsConfig().page_size_failure_threshold_in_kb * 1024; + String limitStr = "Applied page size limit of " + FBUtilities.prettyPrintMemory(bytesLimit); + ClientWarn.instance.warn(limitStr); + logger.trace(limitStr); + limits = limits.forPaging(PageSize.inBytes(bytesLimit)); + } - return DataLimits.cqlLimits(cqlRowLimit, cqlPerPartitionLimit); + return limits; } /** @@ -725,7 +781,7 @@ public int getPerPartitionLimit(QueryOptions options) private int getLimit(Term limit, QueryOptions options) { - int userLimit = DataLimits.NO_LIMIT; + int userLimit = NO_LIMIT; if (limit != null) { @@ -748,12 +804,12 @@ private int getLimit(Term limit, QueryOptions options) return userLimit; } - private NavigableSet> getRequestedRows(QueryOptions options) throws InvalidRequestException + private NavigableSet> getRequestedRows(QueryOptions options, QueryState queryState) throws InvalidRequestException { // Note: getRequestedColumns don't handle static columns, but due to CASSANDRA-5762 // we always do a slice for CQL3 tables, so it's ok to ignore them here assert !restrictions.isColumnRange(); - return restrictions.getClusteringColumns(options); + return restrictions.getClusteringColumns(options, queryState); } /** @@ -975,7 +1031,9 @@ public SelectStatement prepare(boolean forView) throws InvalidRequestException orderingComparator = Collections.reverseOrder(orderingComparator); } - checkNeedsFiltering(restrictions); + checkDisjunctionIsSupported(table, restrictions); + + checkNeedsFiltering(table, restrictions); return new SelectStatement(table, bindVariables, @@ -1064,13 +1122,13 @@ private StatementRestrictions prepareRestrictions(TableMetadata metadata, boolean selectsOnlyStaticColumns, boolean forView) throws InvalidRequestException { - return new StatementRestrictions(StatementType.SELECT, - metadata, - whereClause, - boundNames, - selectsOnlyStaticColumns, - parameters.allowFiltering, - forView); + return StatementRestrictions.create(StatementType.SELECT, + metadata, + whereClause, + boundNames, + selectsOnlyStaticColumns, + parameters.allowFiltering, + forView); } /** Returns a Term for the limit or null if no limit is set */ @@ -1233,16 +1291,29 @@ private boolean isReversed(TableMetadata table, Map ord return isReversed; } + /** + * This verifies that if the expression contains a disjunction - "value = 1 or value = 2" or "value in (1, 2)" + * the indexes involved in the query support disjunction. + */ + private void checkDisjunctionIsSupported(TableMetadata table, StatementRestrictions restrictions) throws InvalidRequestException + { + if (restrictions.usesSecondaryIndexing()) + if (restrictions.needsDisjunctionSupport(table)) + restrictions.throwsRequiresIndexSupportingDisjunctionError(table); + } + /** If ALLOW FILTERING was not specified, this verifies that it is not needed */ - private void checkNeedsFiltering(StatementRestrictions restrictions) throws InvalidRequestException + private void checkNeedsFiltering(TableMetadata table, StatementRestrictions restrictions) throws InvalidRequestException { // non-key-range non-indexed queries cannot involve filtering underneath if (!parameters.allowFiltering && (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing())) { - // We will potentially filter data if either: - // - Have more than one IndexExpression - // - Have no index expression and the row filter is not the identity - checkFalse(restrictions.needFiltering(), StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); + // We will potentially filter data if the row filter is not the identity and there isn't any index group + // supporting all the expressions in the filter. + if (restrictions.needFiltering(table)) + { + restrictions.throwRequiresAllowFilteringError(table); + } } } diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java index 206d11697f95..6b6b99f8b9dc 100644 --- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java @@ -26,6 +26,7 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.exceptions.*; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ClientState; @@ -52,8 +53,11 @@ public void authorize(ClientState state) throws InvalidRequestException, Unautho state.ensureTablePermission(keyspace(), name(), Permission.MODIFY); } - public void validate(ClientState state) throws InvalidRequestException + @Override + public void validate(QueryState state) throws InvalidRequestException { + Guardrails.truncateTableEnabled.ensureEnabled(state); + Schema.instance.validateTable(keyspace(), name()); } diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java index f67db14ea949..3b000362e106 100644 --- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java @@ -176,13 +176,13 @@ protected ModificationStatement prepareInternal(TableMetadata metadata, boolean applyOnlyToStaticColumns = !hasClusteringColumnsSet && appliesOnlyToStaticColumns(operations, conditions); - StatementRestrictions restrictions = new StatementRestrictions(type, - metadata, - whereClause.build(), - bindVariables, - applyOnlyToStaticColumns, - false, - false); + StatementRestrictions restrictions = StatementRestrictions.create(type, + metadata, + whereClause.build(), + bindVariables, + applyOnlyToStaticColumns, + false, + false); return new UpdateStatement(type, bindVariables, @@ -244,13 +244,13 @@ protected ModificationStatement prepareInternal(TableMetadata metadata, boolean applyOnlyToStaticColumns = !hasClusteringColumnsSet && appliesOnlyToStaticColumns(operations, conditions); - StatementRestrictions restrictions = new StatementRestrictions(type, - metadata, - whereClause.build(), - bindVariables, - applyOnlyToStaticColumns, - false, - false); + StatementRestrictions restrictions = StatementRestrictions.create(type, + metadata, + whereClause.build(), + bindVariables, + applyOnlyToStaticColumns, + false, + false); return new UpdateStatement(type, bindVariables, diff --git a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java index 3013d9f997e8..8220ee2f5a7c 100644 --- a/src/java/org/apache/cassandra/cql3/statements/UseStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/UseStatement.java @@ -48,7 +48,8 @@ public void authorize(ClientState state) throws UnauthorizedException state.validateLogin(); } - public void validate(ClientState state) throws InvalidRequestException + @Override + public void validate(QueryState state) throws InvalidRequestException { } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java index 161c9c4a93ed..2a254f213494 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterSchemaStatement.java @@ -29,7 +29,6 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.schema.*; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; -import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.ClientWarn; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.Event.SchemaChange; @@ -44,7 +43,8 @@ protected AlterSchemaStatement(String keyspaceName) this.keyspaceName = keyspaceName; } - public final void validate(ClientState state) + @Override + public void validate(QueryState state) { // no-op; validation is performed while executing the statement, in apply() } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java index 5e3bfa22a5e0..0cba90607b4e 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java @@ -50,9 +50,11 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.DroppedColumn; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableParams; @@ -60,11 +62,13 @@ import org.apache.cassandra.schema.Views; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; import org.apache.cassandra.transport.Event.SchemaChange; import org.apache.cassandra.transport.Event.SchemaChange.Change; import org.apache.cassandra.transport.Event.SchemaChange.Target; import org.apache.cassandra.utils.NoSpamLogger; +import org.apache.cassandra.transport.messages.ResultMessage; import static java.lang.String.format; import static java.lang.String.join; @@ -101,6 +105,11 @@ public Keyspaces apply(Keyspaces schema) throws UnknownHostException return schema.withAddedOrUpdated(apply(keyspace, table)); } + public ResultMessage execute(QueryState state, boolean locally) + { + return super.execute(state, locally); + } + SchemaChange schemaChangeEvent(KeyspacesDiff diff) { return new SchemaChange(Change.UPDATED, Target.TABLE, keyspaceName, tableName); @@ -160,9 +169,20 @@ private static class Column this.type = type; this.isStatic = isStatic; } + } private final Collection newColumns; + private QueryState queryState; + + @Override + public void validate(QueryState state) + { + super.validate(state); + + // save the query state to use it for guardrails validation in #apply + this.queryState = state; + } private AddColumns(String keyspaceName, String tableName, Collection newColumns) { @@ -178,6 +198,8 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table) TableMetadata tableMetadata = tableBuilder.build(); tableMetadata.validate(); + Guardrails.columnsPerTable.guard(tableBuilder.numColumns(), tableName, false, queryState); + return keyspace.withSwapped(keyspace.tables.withSwapped(tableMetadata)) .withSwapped(viewsBuilder.build()); } @@ -388,6 +410,15 @@ private AlterOptions(String keyspaceName, String tableName, TableAttributes attr this.attrs = attrs; } + @Override + public void validate(QueryState state) + { + super.validate(state); + + Guardrails.disallowedTableProperties.ensureAllowed(attrs.updatedProperties(), state); + Guardrails.ignoredTableProperties.maybeIgnoreAndWarn(attrs.updatedProperties(), attrs::removeProperty, state); + } + public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table) { attrs.validate(); @@ -412,7 +443,11 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table) throw ire("read_repair must be set to 'NONE' for transiently replicated keyspaces"); } - return keyspace.withSwapped(keyspace.tables.withSwapped(table.withSwapped(params))); + TableMetadata.Builder builder = table.unbuild().params(params); + for (DroppedColumn.Raw record : attrs.droppedColumnRecords()) + builder.recordColumnDrop(record.prepare(keyspaceName, tableName)); + + return keyspace.withSwapped(keyspace.tables.withSwapped(builder.build())); } } @@ -529,7 +564,7 @@ private enum Kind // DROP private final Set droppedColumns = new HashSet<>(); - private Long timestamp = null; // will use execution timestamp if not provided by query + private Long dropTimestamp = null; // will use execution timestamp if not provided by query // RENAME private final Map renamedColumns = new HashMap<>(); @@ -551,7 +586,7 @@ public AlterTableStatement prepare(ClientState state) { case ALTER_COLUMN: return new AlterColumn(keyspaceName, tableName); case ADD_COLUMNS: return new AddColumns(keyspaceName, tableName, addedColumns); - case DROP_COLUMNS: return new DropColumns(keyspaceName, tableName, droppedColumns, timestamp); + case DROP_COLUMNS: return new DropColumns(keyspaceName, tableName, droppedColumns, dropTimestamp); case RENAME_COLUMNS: return new RenameColumns(keyspaceName, tableName, renamedColumns); case ALTER_OPTIONS: return new AlterOptions(keyspaceName, tableName, attrs); case DROP_COMPACT_STORAGE: return new DropCompactStorage(keyspaceName, tableName); @@ -582,9 +617,9 @@ public void dropCompactStorage() kind = Kind.DROP_COMPACT_STORAGE; } - public void timestamp(long timestamp) + public void dropTimestamp(long timestamp) { - this.timestamp = timestamp; + this.dropTimestamp = timestamp; } public void rename(ColumnIdentifier from, ColumnIdentifier to) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java index a9887c499b3a..47081fa012b0 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java @@ -29,10 +29,12 @@ import org.apache.cassandra.cql3.*; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.Event.SchemaChange; import org.apache.cassandra.transport.Event.SchemaChange.Change; import org.apache.cassandra.transport.Event.SchemaChange.Target; @@ -97,6 +99,7 @@ private static final class AddField extends AlterTypeStatement { private final FieldIdentifier fieldName; private final CQL3Type.Raw type; + private QueryState state; private AddField(String keyspaceName, String typeName, FieldIdentifier fieldName, CQL3Type.Raw type) { @@ -105,6 +108,15 @@ private AddField(String keyspaceName, String typeName, FieldIdentifier fieldName this.type = type; } + @Override + public void validate(QueryState state) + { + super.validate(state); + + // save the query state to use it for guardrails validation in #apply + this.state = state; + } + UserType apply(KeyspaceMetadata keyspace, UserType userType) { if (userType.fieldPosition(fieldName) >= 0) @@ -125,6 +137,9 @@ UserType apply(KeyspaceMetadata keyspace, UserType userType) List fieldNames = new ArrayList<>(userType.fieldNames()); fieldNames.add(fieldName); List> fieldTypes = new ArrayList<>(userType.fieldTypes()); fieldTypes.add(fieldType); + int newSize = userType.size() + 1; + Guardrails.fieldsPerUDT.guard(newSize, userType.getNameAsString(), false, state); + return new UserType(keyspaceName, userType.name, fieldNames, fieldTypes, true); } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java index 1931bb489df3..5247c6bbdab4 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterViewStatement.java @@ -22,9 +22,11 @@ import org.apache.cassandra.auth.Permission; import org.apache.cassandra.cql3.CQLStatement; import org.apache.cassandra.cql3.QualifiedName; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.*; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.Event.SchemaChange; import org.apache.cassandra.transport.Event.SchemaChange.Change; import org.apache.cassandra.transport.Event.SchemaChange.Target; @@ -33,6 +35,7 @@ public final class AlterViewStatement extends AlterSchemaStatement { private final String viewName; private final TableAttributes attrs; + private QueryState state; public AlterViewStatement(String keyspaceName, String viewName, TableAttributes attrs) { @@ -41,6 +44,14 @@ public AlterViewStatement(String keyspaceName, String viewName, TableAttributes this.attrs = attrs; } + public void validate(QueryState state) + { + super.validate(state); + + // save the query state to use it for guardrails validation in #apply + this.state = state; + } + public Keyspaces apply(Keyspaces schema) { KeyspaceMetadata keyspace = schema.getNullable(keyspaceName); @@ -54,6 +65,9 @@ public Keyspaces apply(Keyspaces schema) attrs.validate(); + Guardrails.disallowedTableProperties.ensureAllowed(attrs.updatedProperties(), state); + Guardrails.ignoredTableProperties.maybeIgnoreAndWarn(attrs.updatedProperties(), attrs::removeProperty, state); + TableParams params = attrs.asAlteredTableParams(view.metadata.params); if (params.gcGraceSeconds == 0) @@ -66,8 +80,9 @@ public Keyspaces apply(Keyspaces schema) if (params.defaultTimeToLive > 0) { throw ire("Cannot set or alter default_time_to_live for a materialized view. " + - "Data in a materialized view always expire at the same time than " + - "the corresponding data in the parent table."); + "Data in a materialized view always expires at the same time as " + + "the corresponding data in the parent table. default_time_to_live " + + "must be set to zero, see CASSANDRA-12868 for more information."); } ViewMetadata newView = view.copy(view.metadata.withSwapped(params)); diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java index d5014236a791..c781480a538d 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateIndexStatement.java @@ -18,7 +18,9 @@ package org.apache.cassandra.cql3.statements.schema; import java.util.*; +import java.util.stream.StreamSupport; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; @@ -33,10 +35,13 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.marshal.MapType; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.guardrails.Guardrail; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.index.sasi.SASIIndex; import org.apache.cassandra.schema.*; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.Event.SchemaChange; import org.apache.cassandra.transport.Event.SchemaChange.Change; import org.apache.cassandra.transport.Event.SchemaChange.Target; @@ -51,6 +56,20 @@ public final class CreateIndexStatement extends AlterSchemaStatement private final List rawIndexTargets; private final IndexAttributes attrs; private final boolean ifNotExists; + private QueryState state; + + private static final String DSE_INDEX_WARNING = "Index %s was not created. DSE custom index (%s) is not " + + "supported. Consult the docs on alternatives (SAI indexes, " + + "Secondary Indexes)."; + + @VisibleForTesting + public static final Set DSE_INDEXES = ImmutableSet.of( + "com.datastax.bdp.cassandra.index.solr.SolrSecondaryIndex", + "com.datastax.bdp.cassandra.index.solr.ThriftSolrSecondaryIndex", + "com.datastax.bdp.cassandra.index.solr.Cql3SolrSecondaryIndex", + "com.datastax.bdp.search.solr.ThriftSolrSecondaryIndex", + "com.datastax.bdp.search.solr.Cql3SolrSecondaryIndex" + ); public CreateIndexStatement(String keyspaceName, String tableName, @@ -67,8 +86,24 @@ public CreateIndexStatement(String keyspaceName, this.ifNotExists = ifNotExists; } + @Override + public void validate(QueryState state) + { + super.validate(state); + + // save the query state to use it for guardrails validation in #apply + this.state = state; + } + public Keyspaces apply(Keyspaces schema) { + if (isDseIndexCreateStatement()) + { + // DSE indexes are not supported. The index is not created, the attempt is ignored (doesn't cause error), + // a meaningfull warning is returned instead. + return schema; + } + attrs.validate(); if (attrs.isCustom && attrs.customClass.equals(SASIIndex.class.getName()) && !DatabaseDescriptor.getEnableSASIIndexes()) @@ -115,16 +150,37 @@ public Keyspaces apply(Keyspaces schema) throw ire("Duplicate column '%s' in index target list", target.column); } - indexTargets.forEach(t -> validateIndexTarget(table, t)); + IndexMetadata.Kind kind = attrs.isCustom ? IndexMetadata.Kind.CUSTOM : IndexMetadata.Kind.COMPOSITES; + + indexTargets.forEach(t -> validateIndexTarget(table, kind, t)); String name = null == indexName ? generateIndexName(keyspace, indexTargets) : indexName; - IndexMetadata.Kind kind = attrs.isCustom ? IndexMetadata.Kind.CUSTOM : IndexMetadata.Kind.COMPOSITES; - Map options = attrs.isCustom ? attrs.getOptions() : Collections.emptyMap(); IndexMetadata index = IndexMetadata.fromIndexTargets(indexTargets, name, kind, options); + String className = index.getIndexClassName(); + IndexGuardrails guardRails = IndexGuardrails.forClassName(className); + String indexDescription = indexName == null ? String.format("on table %s", table.name) : String.format("%s on table %s", indexName, table.name); + + // Guardrail to limit number of secondary indexes (per table) + if (guardRails.hasPerTableThreshold()) + { + long indexesOnSameTable = table.indexes.stream().filter(other -> className.equals(other.getIndexClassName())).count(); + guardRails.perTableThreshold.guard(indexesOnSameTable + 1, indexDescription,false, state); + } + + // Guardrail to limit number of secondary indexes (total) + if (guardRails.hasTotalThreshold()) + { + long indexesOnAllTables = StreamSupport.stream(Keyspace.all().spliterator(), false).flatMap(ks -> ks.getColumnFamilyStores().stream()) + .flatMap(ks -> ks.indexManager.listIndexes().stream()) + .map(i -> i.getIndexMetadata().getIndexClassName()) + .filter(otherClassName -> className.equals(otherClassName)).count(); + guardRails.totalThreshold.guard(indexesOnAllTables + 1, indexDescription, false, state); + } + // check to disallow creation of an index which duplicates an existing one in all but name IndexMetadata equalIndex = tryFind(table.indexes, i -> i.equalsWithoutName(index)).orNull(); if (null != equalIndex) @@ -147,16 +203,28 @@ Set clientWarnings(KeyspacesDiff diff) if (attrs.isCustom && attrs.customClass.equals(SASIIndex.class.getName())) return ImmutableSet.of(SASIIndex.USAGE_WARNING); + if (isDseIndexCreateStatement()) + return ImmutableSet.of(String.format(DSE_INDEX_WARNING, indexName, attrs.customClass)); + return ImmutableSet.of(); } - private void validateIndexTarget(TableMetadata table, IndexTarget target) + private boolean isDseIndexCreateStatement() + { + return DSE_INDEXES.contains(attrs.customClass); + } + + private void validateIndexTarget(TableMetadata table, IndexMetadata.Kind kind, IndexTarget target) { ColumnMetadata column = table.getColumn(target.column); if (null == column) throw ire("Column '%s' doesn't exist", target.column); + if ((kind == IndexMetadata.Kind.CUSTOM) && !SchemaConstants.isValidName(target.column.toString())) + throw ire("Column '%s' is longer than the permissible name length of %d characters or" + + " contains non-alphanumeric-underscore characters", target.column, SchemaConstants.NAME_LENGTH); + if (column.type.referencesDuration()) { if (column.type.isCollection()) @@ -265,4 +333,47 @@ public CreateIndexStatement prepare(ClientState state) return new CreateIndexStatement(keyspaceName, tableName.getName(), indexName.getName(), rawIndexTargets, attrs, ifNotExists); } } + + enum IndexGuardrails + { + LEGACY(Guardrails.secondaryIndexesPerTable, null), + SAI(Guardrails.indexesPerTableSai, Guardrails.indexesTotalSai), + SASI(Guardrails.indexesPerTableSasi, null), + UNKNOWN(null, null); + + final Guardrail.Threshold perTableThreshold; + final Guardrail.Threshold totalThreshold; + + IndexGuardrails(Guardrail.Threshold perTableThreshold, Guardrail.Threshold totalThreshold) + { + this.perTableThreshold = perTableThreshold; + this.totalThreshold = totalThreshold; + } + + boolean hasPerTableThreshold() + { + return perTableThreshold != null; + } + + boolean hasTotalThreshold() + { + return totalThreshold != null; + } + + static IndexGuardrails forClassName(String className) + { + switch (className) + { + case "org.apache.cassandra.index.internal.CassandraIndex": + return IndexGuardrails.LEGACY; + case "org.apache.cassandra.index.sasi.SASIIndex": + return IndexGuardrails.SASI; + case "org.apache.cassandra.index.sai.StorageAttachedIndex": + return IndexGuardrails.SAI; + default: + return IndexGuardrails.UNKNOWN; + } + } + + } } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java index 806b50a2bdb0..2dab938bcd60 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java @@ -133,6 +133,11 @@ Set clientWarnings(KeyspacesDiff diff) clientWarnings.add(msg); } + if (attrs.hasProperty("graph_engine")) + { + clientWarnings.add("The unsupported graph property 'graph_engine' was ignored."); + } + return clientWarnings; } diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java index 1339ba39f7b1..0078af4289bc 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java @@ -33,19 +33,21 @@ import org.apache.cassandra.auth.Permission; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.*; +import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.marshal.*; import org.apache.cassandra.exceptions.AlreadyExistsException; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.*; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; import org.apache.cassandra.transport.Event.SchemaChange; import org.apache.cassandra.transport.Event.SchemaChange.Change; import org.apache.cassandra.transport.Event.SchemaChange.Target; -import static java.util.Comparator.comparing; - import static com.google.common.collect.Iterables.concat; +import static java.util.Comparator.comparing; public final class CreateTableStatement extends AlterSchemaStatement { @@ -92,6 +94,37 @@ public CreateTableStatement(String keyspaceName, this.useCompactStorage = useCompactStorage; } + @Override + public void validate(QueryState state) + { + super.validate(state); + + // Some tools use CreateTableStatement, and the guardrails below both don't make too much sense for tools and + // require the server to be initialized, so skipping them if it isn't. + if (Guardrails.ready()) + { + // Guardrails on table properties + Guardrails.disallowedTableProperties.ensureAllowed(attrs.updatedProperties(), state); + Guardrails.ignoredTableProperties.maybeIgnoreAndWarn(attrs.updatedProperties(), attrs::removeProperty, state); + + // Guardrail on counter + if (rawColumns.values().stream().anyMatch(CQL3Type.Raw::isCounter)) + Guardrails.counterEnabled.ensureEnabled(state); + + // Guardrail on columns per table + Guardrails.columnsPerTable.guard(rawColumns.size(), tableName, false, state); + + if (Guardrails.tablesLimit.enabled(state)) + { + // guardrails on number of tables + int totalUserTables = Schema.instance.getNonInternalKeyspaces().stream().map(Keyspace::open) + .mapToInt(keyspace -> keyspace.getColumnFamilyStores().size()) + .sum(); + Guardrails.tablesLimit.guard(totalUserTables + 1, tableName, false, state); + } + } + } + public Keyspaces apply(Keyspaces schema) { KeyspaceMetadata keyspace = schema.getNullable(keyspaceName); @@ -291,6 +324,8 @@ public TableMetadata.Builder builder(Types types) builder.addRegularColumn(column, type.getType()); }); } + for (DroppedColumn.Raw record : attrs.droppedColumnRecords()) + builder.recordColumnDrop(record.prepare(keyspaceName, tableName)); return builder; } @@ -372,6 +407,8 @@ else if (!builder.hasRegularColumns()) @Override public Set clientWarnings(KeyspacesDiff diff) { + ImmutableSet.Builder warnings = ImmutableSet.builder(); + int tableCount = Schema.instance.getNumberOfTables(); if (tableCount > DatabaseDescriptor.tableCountWarnThreshold()) { @@ -379,9 +416,35 @@ public Set clientWarnings(KeyspacesDiff diff) tableCount, Schema.instance.getKeyspaces().size()); logger.warn(msg); - return ImmutableSet.of(msg); + warnings.add(msg); } - return ImmutableSet.of(); + + if (attrs.hasUnsupportedDseCompaction()) + { + Map compactionOptions = attrs.getMap(TableParams.Option.COMPACTION.toString()); + String strategy = compactionOptions.get(CompactionParams.Option.CLASS.toString()); + warnings.add(String.format("The given compaction strategy (%s) is not supported. ", strategy) + + "The compaction strategy parameter was overridden with the default " + + String.format("(%s). ", CompactionParams.DEFAULT.klass().getCanonicalName()) + + "Inspect your schema and adjust other table properties if needed."); + } + + if (attrs.hasProperty("nodesync")) + { + warnings.add("The unsupported 'nodesync' table option was ignored."); + } + + if (attrs.hasProperty("dse_vertex_label_property")) + { + warnings.add("The unsupported graph table property was ignored (VERTEX LABEL)."); + } + + if (attrs.hasProperty("dse_edge_label_property")) + { + warnings.add("The unsupported graph table property was ignored (EDGE LABEL)."); + } + + return warnings.build(); } private static class DefaultNames @@ -422,11 +485,16 @@ public String defaultCompactValueName() } public static TableMetadata.Builder parse(String cql, String keyspace) + { + return parse(cql, keyspace, Types.none()); + } + + public static TableMetadata.Builder parse(String cql, String keyspace, Types types) { return CQLFragmentParser.parseAny(CqlParser::createTableStatement, cql, "CREATE TABLE") - .keyspace(keyspace) - .prepare(null) // works around a messy ClientState/QueryProcessor class init deadlock - .builder(Types.none()); + .keyspace(keyspace) + .prepare(null) // works around a messy ClientState/QueryProcessor class init deadlock + .builder(types); } public final static class Raw extends CQLStatement.Raw diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java index 7c1717e24c2e..ef82920e722c 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java @@ -23,16 +23,20 @@ import org.apache.cassandra.audit.AuditLogEntryType; import org.apache.cassandra.auth.Permission; import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.CQLFragmentParser; import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.CqlParser; import org.apache.cassandra.cql3.FieldIdentifier; import org.apache.cassandra.cql3.UTName; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Keyspaces; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.schema.Types; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.Event.SchemaChange; import org.apache.cassandra.transport.Event.SchemaChange.Change; import org.apache.cassandra.transport.Event.SchemaChange.Target; @@ -61,6 +65,14 @@ public CreateTypeStatement(String keyspaceName, this.ifNotExists = ifNotExists; } + @Override + public void validate(QueryState state) + { + super.validate(state); + + Guardrails.fieldsPerUDT.guard(fieldNames.size(), typeName, false, state); + } + public Keyspaces apply(Keyspaces schema) { KeyspaceMetadata keyspace = schema.getNullable(keyspaceName); @@ -120,6 +132,35 @@ public String toString() return String.format("%s (%s, %s)", getClass().getSimpleName(), keyspaceName, typeName); } + public static UserType parse(String cql, String keyspace) + { + return parse(cql, keyspace, Types.none()); + } + + public static UserType parse(String cql, String keyspace, Types userTypes) + { + return CQLFragmentParser.parseAny(CqlParser::createTypeStatement, cql, "CREATE TYPE") + .keyspace(keyspace) + .prepare(null) // works around a messy ClientState/QueryProcessor class init deadlock + .createType(userTypes); + } + + /** + * Build the {@link UserType} this statement creates. + * + * @param existingTypes the user-types existing in the keyspace in which the type is created (and thus on which + * the created type may depend on). + * @return the created type. + */ + public UserType createType(Types existingTypes) + { + List> fieldTypes = rawFieldTypes.stream() + .map(t -> t.prepare(keyspaceName, existingTypes).getType()) + .collect(toList()); + UserType type = new UserType(keyspaceName, bytes(typeName), fieldNames, fieldTypes, true); + return type; + } + public static final class Raw extends CQLStatement.Raw { private final UTName name; @@ -134,6 +175,12 @@ public Raw(UTName name, boolean ifNotExists) this.ifNotExists = ifNotExists; } + public Raw keyspace(String keyspace) + { + name.setKeyspace(keyspace); + return this; + } + public CreateTypeStatement prepare(ClientState state) { String keyspaceName = name.hasKeyspace() ? name.getKeyspace() : state.getKeyspace(); diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java index 7e51eb2ad818..10c0d6c83e85 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateViewStatement.java @@ -18,6 +18,8 @@ package org.apache.cassandra.cql3.statements.schema; import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; @@ -36,9 +38,11 @@ import org.apache.cassandra.db.view.View; import org.apache.cassandra.exceptions.AlreadyExistsException; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.schema.*; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.Event.SchemaChange; import org.apache.cassandra.transport.Event.SchemaChange.Change; import org.apache.cassandra.transport.Event.SchemaChange.Target; @@ -64,6 +68,7 @@ public final class CreateViewStatement extends AlterSchemaStatement private final TableAttributes attrs; private final boolean ifNotExists; + private QueryState state; public CreateViewStatement(String keyspaceName, String tableName, @@ -96,6 +101,15 @@ public CreateViewStatement(String keyspaceName, this.ifNotExists = ifNotExists; } + @Override + public void validate(QueryState state) + { + super.validate(state); + + // save the query state to use it for guardrails validation in #apply + this.state = state; + } + public Keyspaces apply(Keyspaces schema) { if (!DatabaseDescriptor.getEnableMaterializedViews()) @@ -137,6 +151,18 @@ public Keyspaces apply(Keyspaces schema) if (table.isView()) throw ire("Materialized views cannot be created against other materialized views"); + // Guardrails on table properties + Guardrails.disallowedTableProperties.ensureAllowed(attrs.updatedProperties(), state); + Guardrails.ignoredTableProperties.maybeIgnoreAndWarn(attrs.updatedProperties(), attrs::removeProperty, state); + + // guardrails to limit number of mvs per table. + Set baseTableViews = StreamSupport.stream(keyspace.views.forTable(table.id).spliterator(), false) + .collect(Collectors.toCollection(HashSet::new)); + Guardrails.materializedViewsPerTable.guard(baseTableViews.size() + 1, + String.format("%s on table %s", viewName, table.name), + false, + state); + if (table.params.gcGraceSeconds == 0) { throw ire("Cannot create materialized view '%s' for base table " + @@ -246,15 +272,14 @@ public Keyspaces apply(Keyspaces schema) if (whereClause.containsCustomExpressions()) throw ire("WHERE clause for materialized view '%s' cannot contain custom index expressions", viewName); - StatementRestrictions restrictions = - new StatementRestrictions(StatementType.SELECT, - table, - whereClause, - VariableSpecifications.empty(), - false, - false, - true, - true); + StatementRestrictions restrictions = StatementRestrictions.create(StatementType.SELECT, + table, + whereClause, + VariableSpecifications.empty(), + false, + false, + true, + true); List nonRestrictedPrimaryKeyColumns = Lists.newArrayList(filter(primaryKeyColumns, name -> !restrictions.isRestricted(table.getColumn(name)))); diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java index 42fcaf4e69e8..88764a93f1a7 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/KeyspaceAttributes.java @@ -38,7 +38,7 @@ public final class KeyspaceAttributes extends PropertyDefinitions for (Option option : Option.values()) validBuilder.add(option.toString()); validKeywords = validBuilder.build(); - obsoleteKeywords = ImmutableSet.of(); + obsoleteKeywords = ImmutableSet.of("graph_engine"); } public void validate() diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java index 126e6d7857b1..13344b1de450 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/TableAttributes.java @@ -17,17 +17,25 @@ */ package org.apache.cassandra.cql3.statements.schema; +import java.util.Collection; +import java.util.HashMap; import java.util.Map; import java.util.Set; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.statements.PropertyDefinitions; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.schema.DroppedColumn; +import org.apache.cassandra.schema.MemtableParams; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.schema.TableParams.Option; @@ -39,8 +47,19 @@ public final class TableAttributes extends PropertyDefinitions { public static final String ID = "id"; - private static final Set validKeywords; - private static final Set obsoleteKeywords; + public static final Set validKeywords; + private static final Set obsoleteKeywords = ImmutableSet.of( + "nodesync", + "dse_vertex_label_property", + "dse_edge_label_property" + ); + + private static final Set UNSUPPORTED_DSE_COMPACTION_STRATEGIES = ImmutableSet.of( + "org.apache.cassandra.db.compaction.TieredCompactionStrategy", + "TieredCompactionStrategy", + "org.apache.cassandra.db.compaction.MemoryOnlyStrategy", + "MemoryOnlyStrategy" + ); static { @@ -49,15 +68,28 @@ public final class TableAttributes extends PropertyDefinitions validBuilder.add(option.toString()); validBuilder.add(ID); validKeywords = validBuilder.build(); - obsoleteKeywords = ImmutableSet.of(); } + private final Map droppedColumnRecords = new HashMap<>(); + public void validate() { validate(validKeywords, obsoleteKeywords); build(TableParams.builder()).validate(); } + public void addDroppedColumnRecord(ColumnIdentifier name, CQL3Type.Raw type, boolean isStatic, long timestamp) + { + DroppedColumn.Raw newRecord = new DroppedColumn.Raw(name, type, isStatic, timestamp); + if (droppedColumnRecords.put(name, newRecord) != null) + throw new InvalidRequestException(String.format("Cannot have multiple dropped column record for column %s", name)); + } + + public Collection droppedColumnRecords() + { + return droppedColumnRecords.values(); + } + TableParams asNewTableParams() { return build(TableParams.builder()); @@ -83,6 +115,29 @@ public TableId getId() throws ConfigurationException } } + /** + * Returs `true` if this attributes instance has a COMPACTION option with a recognized unsupported compaction + * strategy class (coming from DSE). `false` otherwise. + */ + boolean hasUnsupportedDseCompaction() + { + if (hasOption(Option.COMPACTION)) + { + Map compactionOptions = getMap(Option.COMPACTION); + String strategy = compactionOptions.get(CompactionParams.Option.CLASS.toString()); + return UNSUPPORTED_DSE_COMPACTION_STRATEGIES.contains(strategy); + } + else + { + return false; + } + } + + public static Set allKeywords() + { + return Sets.union(validKeywords, obsoleteKeywords); + } + private TableParams build(TableParams.Builder builder) { if (hasOption(Option.BLOOM_FILTER_FP_CHANCE)) @@ -95,7 +150,12 @@ private TableParams build(TableParams.Builder builder) builder.comment(getString(Option.COMMENT)); if (hasOption(Option.COMPACTION)) - builder.compaction(CompactionParams.fromMap(getMap(Option.COMPACTION))); + { + if (hasUnsupportedDseCompaction()) + builder.compaction(CompactionParams.DEFAULT); + else + builder.compaction(CompactionParams.fromMap(getMap(Option.COMPACTION))); + } if (hasOption(Option.COMPRESSION)) { @@ -110,6 +170,9 @@ private TableParams build(TableParams.Builder builder) builder.compression(CompressionParams.fromMap(getMap(Option.COMPRESSION))); } + if (hasOption(Option.MEMTABLE)) + builder.memtable(MemtableParams.fromMap(getMap(Option.MEMTABLE))); + if (hasOption(Option.DEFAULT_TIME_TO_LIVE)) builder.defaultTimeToLive(getInt(Option.DEFAULT_TIME_TO_LIVE)); diff --git a/src/java/org/apache/cassandra/db/AbstractArrayClusteringPrefix.java b/src/java/org/apache/cassandra/db/AbstractArrayClusteringPrefix.java index 211eeb0460cd..ee3e43e5782e 100644 --- a/src/java/org/apache/cassandra/db/AbstractArrayClusteringPrefix.java +++ b/src/java/org/apache/cassandra/db/AbstractArrayClusteringPrefix.java @@ -49,7 +49,7 @@ public ByteBuffer[] getBufferArray() return out; } - public ClusteringPrefix minimize() + public ClusteringPrefix retainable() { return this; } diff --git a/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java b/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java index 457d0c4befa8..f8ffcaee8eec 100644 --- a/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java +++ b/src/java/org/apache/cassandra/db/AbstractBufferClusteringPrefix.java @@ -42,10 +42,18 @@ public ByteBuffer[] getBufferArray() return getRawValues(); } - public ClusteringPrefix minimize() + public ClusteringPrefix retainable() { if (!ByteBufferUtil.canMinimize(values)) return this; - return new BufferClustering(ByteBufferUtil.minimizeBuffers(values)); + + ByteBuffer[] values = ByteBufferUtil.minimizeBuffers(this.values); + if (kind.isBoundary()) + return accessor().factory().boundary(kind, values); + if (kind.isBound()) + return accessor().factory().bound(kind, values); + + assert kind() != Kind.STATIC_CLUSTERING; // not minimizable + return accessor().factory().clustering(values); } } diff --git a/src/java/org/apache/cassandra/db/BufferDecoratedKey.java b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java index d375162240d7..07f610fd7ca4 100644 --- a/src/java/org/apache/cassandra/db/BufferDecoratedKey.java +++ b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java @@ -19,7 +19,9 @@ import java.nio.ByteBuffer; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; public class BufferDecoratedKey extends DecoratedKey { @@ -36,4 +38,34 @@ public ByteBuffer getKey() { return key; } + + @Override + public int getKeyLength() + { + return key.remaining(); + } + + /** + * A factory method that translates the given byte-comparable representation to a {@link BufferDecoratedKey} + * instance. If the given byte comparable doesn't represent the encoding of a buffer decorated key, anything from a + * wide variety of throwables may be thrown (e.g. {@link AssertionError}, {@link IndexOutOfBoundsException}, + * {@link IllegalStateException}, etc.). + * + * @param byteComparable A byte-comparable representation (presumably of a {@link BufferDecoratedKey} instance). + * @param version The encoding version used for the given byte comparable. + * @param partitioner The partitioner of the encoded decorated key. Needed in order to correctly decode the token + * bytes of the key. + * @return A new {@link BufferDecoratedKey} instance, corresponding to the given byte-comparable representation. If + * we were to call {@link #asComparableBytes(Version)} on the returned object, we should get a {@link ByteSource} + * equal to the one of the input byte comparable. + */ + public static BufferDecoratedKey fromByteComparable(ByteComparable byteComparable, + Version version, + IPartitioner partitioner) + { + return DecoratedKey.fromByteComparable(byteComparable, + version, + partitioner, + (token, keyBytes) -> new BufferDecoratedKey(token, ByteBuffer.wrap(keyBytes))); + } } diff --git a/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java b/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java index efba11f1a4ff..570a2898d1cf 100644 --- a/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java +++ b/src/java/org/apache/cassandra/db/CassandraKeyspaceWriteHandler.java @@ -18,9 +18,13 @@ package org.apache.cassandra.db; +import java.util.HashSet; +import java.util.Set; + import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.commitlog.CommitLogPosition; import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.schema.TableId; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.concurrent.OpOrder; @@ -46,8 +50,7 @@ public WriteContext beginWrite(Mutation mutation, boolean makeDurable) throws Re CommitLogPosition position = null; if (makeDurable) { - Tracing.trace("Appending to commitlog"); - position = CommitLog.instance.add(mutation); + position = addToCommitLog(mutation); } return new CassandraWriteContext(group, position); } @@ -61,6 +64,43 @@ public WriteContext beginWrite(Mutation mutation, boolean makeDurable) throws Re } } + private CommitLogPosition addToCommitLog(Mutation mutation) + { + CommitLogPosition position; + // Usually one of these will be true, so first check if that's the case. + boolean allSkipCommitlog = true; + boolean noneSkipCommitlog = true; + for (TableId id : mutation.getTableIds()) + { + if (keyspace.getColumnFamilyStore(id).writesShouldSkipCommitLog()) + noneSkipCommitlog = false; + else + allSkipCommitlog = false; + } + + if (!noneSkipCommitlog) + { + if (allSkipCommitlog) + return null; + else + { + Set ids = new HashSet<>(); + for (TableId id : mutation.getTableIds()) + { + if (keyspace.getColumnFamilyStore(id).writesShouldSkipCommitLog()) + ids.add(id); + } + mutation = mutation.without(ids); + } + } + // Note: It may be a good idea to precalculate none/all for the set of all tables in the keyspace, + // or memoize the mutation.getTableIds()->ids map (needs invalidation on schema version change). + + Tracing.trace("Appending to commitlog"); + position = CommitLog.instance.add(mutation); + return position; + } + @SuppressWarnings("resource") // group is closed when CassandraWriteContext is closed private WriteContext createEmptyContext() { diff --git a/src/java/org/apache/cassandra/db/CassandraTableWriteHandler.java b/src/java/org/apache/cassandra/db/CassandraTableWriteHandler.java index 146539c26ef4..d9dbfa1e0ead 100644 --- a/src/java/org/apache/cassandra/db/CassandraTableWriteHandler.java +++ b/src/java/org/apache/cassandra/db/CassandraTableWriteHandler.java @@ -33,10 +33,10 @@ public CassandraTableWriteHandler(ColumnFamilyStore cfs) @Override @SuppressWarnings("resource") - public void write(PartitionUpdate update, WriteContext context, UpdateTransaction updateTransaction) + public void write(PartitionUpdate update, WriteContext context, boolean updateIndexes) { CassandraWriteContext ctx = CassandraWriteContext.fromContext(context); Tracing.trace("Adding to {} memtable", update.metadata().name); - cfs.apply(update, updateTransaction, ctx.getGroup(), ctx.getPosition()); + cfs.apply(update, ctx, updateIndexes); } } diff --git a/src/java/org/apache/cassandra/db/Clustering.java b/src/java/org/apache/cassandra/db/Clustering.java index c6856382482b..09cc48d4d320 100644 --- a/src/java/org/apache/cassandra/db/Clustering.java +++ b/src/java/org/apache/cassandra/db/Clustering.java @@ -55,6 +55,18 @@ public default Clustering copy(AbstractAllocator allocator) return new BufferClustering(newValues); } + @Override + default ClusteringBound asStartBound() + { + return ClusteringBound.inclusiveStartOf(this); + } + + @Override + default ClusteringBound asEndBound() + { + return ClusteringBound.inclusiveEndOf(this); + } + public default String toString(TableMetadata metadata) { StringBuilder sb = new StringBuilder(); diff --git a/src/java/org/apache/cassandra/db/ClusteringBound.java b/src/java/org/apache/cassandra/db/ClusteringBound.java index 364856f21229..0eb6a920cdf3 100644 --- a/src/java/org/apache/cassandra/db/ClusteringBound.java +++ b/src/java/org/apache/cassandra/db/ClusteringBound.java @@ -26,15 +26,22 @@ import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.utils.memory.AbstractAllocator; +import static org.apache.cassandra.db.AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY; + /** * The start or end of a range of clusterings, either inclusive or exclusive. */ public interface ClusteringBound extends ClusteringBoundOrBoundary { /** The smallest start bound, i.e. the one that starts before any row. */ - public static final ClusteringBound BOTTOM = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, BufferClusteringBound.EMPTY_VALUES_ARRAY); + public static final ClusteringBound BOTTOM = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, EMPTY_VALUES_ARRAY); /** The biggest end bound, i.e. the one that ends after any row. */ - public static final ClusteringBound TOP = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, BufferClusteringBound.EMPTY_VALUES_ARRAY); + public static final ClusteringBound TOP = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, EMPTY_VALUES_ARRAY); + + /** The biggest start bound, i.e. the one that starts after any row. */ + public static final ClusteringBound MAX_START = new BufferClusteringBound(Kind.EXCL_START_BOUND, EMPTY_VALUES_ARRAY); + /** The smallest end bound, i.e. the one that end before any row. */ + public static final ClusteringBound MIN_END = new BufferClusteringBound(Kind.EXCL_END_BOUND, EMPTY_VALUES_ARRAY); public static ClusteringPrefix.Kind boundKind(boolean isStart, boolean isInclusive) { @@ -69,6 +76,21 @@ default boolean isExclusive() return kind() == Kind.EXCL_START_BOUND || kind() == Kind.EXCL_END_BOUND; } + default boolean isArtificial() + { + return kind() == Kind.SSTABLE_LOWER_BOUND || kind() == Kind.SSTABLE_UPPER_BOUND; + } + + default ClusteringBound artificialLowerBound() + { + return create(Kind.SSTABLE_LOWER_BOUND, this); + } + + default ClusteringBound artificialUpperBound() + { + return create(Kind.SSTABLE_UPPER_BOUND, this); + } + // For use by intersects, it's called with the sstable bound opposite to the slice bound // (so if the slice bound is a start, it's call with the max sstable bound) default int compareTo(ClusteringComparator comparator, List sstableBound) @@ -102,12 +124,12 @@ static ClusteringBound create(ClusteringPrefix.Kind kind, ClusteringPrefi return from.accessor().factory().bound(kind, from.getRawValues()); } - public static ClusteringBound inclusiveStartOf(ClusteringPrefix from) + public static ClusteringBound inclusiveStartOf(ClusteringPrefix from) { return create(ClusteringPrefix.Kind.INCL_START_BOUND, from); } - public static ClusteringBound inclusiveEndOf(ClusteringPrefix from) + public static ClusteringBound inclusiveEndOf(ClusteringPrefix from) { return create(ClusteringPrefix.Kind.INCL_END_BOUND, from); } @@ -134,4 +156,16 @@ public static ClusteringBound create(ClusteringComparator comparator, boolean } return builder.buildBound(isStart, isInclusive); } + + @Override + default ClusteringBound asStartBound() + { + return this; + } + + @Override + default ClusteringBound asEndBound() + { + return this; + } } diff --git a/src/java/org/apache/cassandra/db/ClusteringBoundary.java b/src/java/org/apache/cassandra/db/ClusteringBoundary.java index 3e50f5264ccf..df35a5d1bb14 100644 --- a/src/java/org/apache/cassandra/db/ClusteringBoundary.java +++ b/src/java/org/apache/cassandra/db/ClusteringBoundary.java @@ -37,4 +37,16 @@ public static ClusteringBoundary create(ClusteringBound.Kind kind, Cluste { return from.accessor().factory().boundary(kind, from.getRawValues()); } + + @Override + default ClusteringBound asStartBound() + { + return openBound(false); + } + + @Override + default ClusteringBound asEndBound() + { + return closeBound(false); + } } diff --git a/src/java/org/apache/cassandra/db/ClusteringComparator.java b/src/java/org/apache/cassandra/db/ClusteringComparator.java index fdc450813ff2..81cf57ebfe9b 100644 --- a/src/java/org/apache/cassandra/db/ClusteringComparator.java +++ b/src/java/org/apache/cassandra/db/ClusteringComparator.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -30,7 +31,15 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.serializers.MarshalException; -import org.apache.cassandra.io.sstable.IndexInfo; +import org.apache.cassandra.io.sstable.format.big.IndexInfo; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +import static org.apache.cassandra.utils.bytecomparable.ByteSource.EXCLUDED; +import static org.apache.cassandra.utils.bytecomparable.ByteSource.NEXT_COMPONENT; +import static org.apache.cassandra.utils.bytecomparable.ByteSource.NEXT_COMPONENT_NULL; +import static org.apache.cassandra.utils.bytecomparable.ByteSource.NEXT_COMPONENT_NULL_REVERSED; +import static org.apache.cassandra.utils.bytecomparable.ByteSource.TERMINATOR; /** * A comparator of clustering prefixes (or more generally of {@link Clusterable}}. @@ -232,6 +241,259 @@ public void validate(ClusteringPrefix clustering) } } + /** + * Produce a prefix-free byte-comparable representation of the given value, i.e. such a sequence of bytes that any + * pair x, y of valid values of this type + * compare(x, y) == compareLexicographicallyUnsigned(asByteComparable(x), asByteComparable(y)) + * and + * asByteComparable(x) is not a prefix of asByteComparable(y) + */ + public ByteComparable asByteComparable(ClusteringPrefix clustering) + { + return new ByteComparableClustering<>(clustering); + } + + /** + * A prefix-free byte-comparable representation for a clustering or prefix. + * + * Adds a NEXT_COMPONENT byte before each component (allowing inclusive/exclusive bounds over incomplete prefixes + * of that length) and finishes with a suitable byte for the clustering kind. Also deals with null entries. + * + * Since all types' encodings are weakly prefix-free, this is guaranteed to be prefix-free as long as the + * bound/ClusteringPrefix terminators are different from the separator byte. It is okay for the terminator for + * Clustering to be the same as the separator, as all Clusterings must be completely specified. + * + * See also {@link AbstractType#asComparableBytes}. + * + * Some examples: + * "A", 0005, Clustering -> 40 4100 40 0005 40 + * "B", 0006, InclusiveEnd -> 40 4200 40 0006 60 + * "A", ExclusiveStart -> 40 4100 60 + * "", null, Clustering -> 40 00 3F 40 + * "", 0000, Clustering -> 40 00 40 0000 40 + * BOTTOM -> 20 + */ + private class ByteComparableClustering implements ByteComparable + { + private final ClusteringPrefix src; + + ByteComparableClustering(ClusteringPrefix src) + { + this.src = src; + } + + @Override + public ByteSource asComparableBytes(Version version) + { + return new ByteSource() + { + private ByteSource current = null; + private int srcnum = -1; + + @Override + public int next() + { + if (current != null) + { + int b = current.next(); + if (b > END_OF_STREAM) + return b; + current = null; + } + + int sz = src.size(); + if (srcnum == sz) + return END_OF_STREAM; + + ++srcnum; + if (srcnum == sz) + return src.kind().asByteComparableValue(version); + + current = subtype(srcnum).asComparableBytes(src.accessor(), src.get(srcnum), version); + if (current == null) + return subtype(srcnum).isReversed() ? NEXT_COMPONENT_NULL_REVERSED : NEXT_COMPONENT_NULL; + + return NEXT_COMPONENT; + } + }; + } + + public String toString() + { + return src.clusteringString(subtypes()); + } + } + + /** + * Produces a clustering from the given byte-comparable value. The method will throw an exception if the value + * does not correctly encode a clustering of this type, including if it encodes a position before or after a + * clustering (i.e. a bound/boundary). + * + * @param accessor Accessor to use to construct components. Because this will be used to construct individual + * arrays/buffers for each component, it may be sensible to use an accessor that allocates larger + * buffers in advance. + * @param comparable The clustering encoded as a byte-comparable sequence. + */ + public Clustering clusteringFromByteComparable(ValueAccessor accessor, + ByteComparable comparable) + { + ByteComparable.Version version = ByteComparable.Version.OSS41; + ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version)); + if (orderedBytes == null) + return null; + + // First check for special cases (partition key only, static clustering) that can do without buffers. + int sep = orderedBytes.next(); + switch (sep) + { + case TERMINATOR: + assert size() == 0 : "Terminator should be after " + size() + " components, got 0"; + return accessor.factory().clustering(); + case EXCLUDED: + return accessor.factory().staticClustering(); + default: + // continue with processing + } + + int cc = 0; + V[] components = accessor.createArray(size()); + + while (true) + { + switch (sep) + { + case NEXT_COMPONENT_NULL: + case NEXT_COMPONENT_NULL_REVERSED: + components[cc] = accessor.empty(); + break; + case NEXT_COMPONENT: + components[cc] = subtype(cc).fromComparableBytes(accessor, orderedBytes, version); + break; + case TERMINATOR: + assert cc == size() : "Terminator should be after " + size() + " components, got " + cc; + return accessor.factory().clustering(components); + case EXCLUDED: + throw new AssertionError("Unexpected static terminator after the first component"); + default: + throw new AssertionError("Unexpected separator " + Integer.toHexString(sep) + " in Clustering encoding"); + } + ++cc; + sep = orderedBytes.next(); + } + } + + /** + * Produces a clustering bound from the given byte-comparable value. The method will throw an exception if the value + * does not correctly encode a bound position of this type, including if it encodes an exact clustering. + * + * Note that the encoded clustering position cannot specify the type of bound (i.e. start/end/boundary) because to + * correctly compare clustering positions the encoding must be the same for the different types (e.g. the position + * for a exclusive end and an inclusive start is the same, before the exact clustering). The type must be supplied + * separately (in the bound... vs boundary... call and isEnd argument). + * + * @param accessor Accessor to use to construct components. Because this will be used to construct individual + * arrays/buffers for each component, it may be sensible to use an accessor that allocates larger + * buffers in advance. + * @param comparable The clustering position encoded as a byte-comparable sequence. + * @param isEnd true if the bound marks the end of a range, false is it marks the start. + */ + public ClusteringBound boundFromByteComparable(ValueAccessor accessor, + ByteComparable comparable, + boolean isEnd) + { + ByteComparable.Version version = ByteComparable.Version.OSS41; + ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version)); + if (orderedBytes == null) + return null; + + int sep = orderedBytes.next(); + int cc = 0; + V[] components = accessor.createArray(size()); + + while (true) + { + switch (sep) + { + case NEXT_COMPONENT_NULL: + case NEXT_COMPONENT_NULL_REVERSED: + components[cc] = accessor.empty(); + break; + case NEXT_COMPONENT: + components[cc] = subtype(cc).fromComparableBytes(accessor, orderedBytes, version); + break; + case ByteSource.LT_NEXT_COMPONENT: + return accessor.factory().bound(isEnd ? ClusteringPrefix.Kind.EXCL_END_BOUND + : ClusteringPrefix.Kind.INCL_START_BOUND, + Arrays.copyOf(components, cc)); + case ByteSource.GT_NEXT_COMPONENT: + return accessor.factory().bound(isEnd ? ClusteringPrefix.Kind.INCL_END_BOUND + : ClusteringPrefix.Kind.EXCL_START_BOUND, + Arrays.copyOf(components, cc)); + + case ByteSource.LTLT_NEXT_COMPONENT: + case ByteSource.GTGT_NEXT_COMPONENT: + throw new AssertionError("Unexpected sstable lower/upper bound - byte comparable representation of artificial sstable bounds is not supported"); + + default: + throw new AssertionError("Unexpected separator " + Integer.toHexString(sep) + " in ClusteringBound encoding"); + } + ++cc; + sep = orderedBytes.next(); + } + } + + /** + * Produces a clustering boundary from the given byte-comparable value. The method will throw an exception if the + * value does not correctly encode a bound position of this type, including if it encodes an exact clustering. + * + * Note that the encoded clustering position cannot specify the type of bound (i.e. start/end/boundary) because to + * correctly compare clustering positions the encoding must be the same for the different types (e.g. the position + * for a exclusive end and an inclusive start is the same, before the exact clustering). The type must be supplied + * separately (in the bound... vs boundary... call and isEnd argument). + * + * @param accessor Accessor to use to construct components. Because this will be used to construct individual + * arrays/buffers for each component, it may be sensible to use an accessor that allocates larger + * buffers in advance. + * @param comparable The clustering position encoded as a byte-comparable sequence. + */ + public ClusteringBoundary boundaryFromByteComparable(ValueAccessor accessor, + ByteComparable comparable) + { + ByteComparable.Version version = ByteComparable.Version.OSS41; + ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version)); + if (orderedBytes == null) + return null; + + // First check for special cases (partition key only, static clustering) that can do without buffers. + int sep = orderedBytes.next(); + int cc = 0; + V[] components = accessor.createArray(size()); + + while (true) + { + switch (sep) + { + case NEXT_COMPONENT_NULL: + case NEXT_COMPONENT_NULL_REVERSED: + components[cc] = accessor.empty(); + break; + case NEXT_COMPONENT: + components[cc] = subtype(cc).fromComparableBytes(accessor, orderedBytes, version); + break; + case ByteSource.LT_NEXT_COMPONENT: + return accessor.factory().boundary(ClusteringPrefix.Kind.EXCL_END_INCL_START_BOUNDARY, + Arrays.copyOf(components, cc)); + case ByteSource.GT_NEXT_COMPONENT: + return accessor.factory().boundary(ClusteringPrefix.Kind.INCL_END_EXCL_START_BOUNDARY, + Arrays.copyOf(components, cc)); + default: + throw new AssertionError("Unexpected separator " + Integer.toHexString(sep) + " in ClusteringBoundary encoding"); + } + ++cc; + sep = orderedBytes.next(); + } + } + /** * A comparator for rows. * diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java index a1291c889f1d..59fd83b8cc6a 100644 --- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java +++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; +import java.util.function.ToIntFunction; import org.apache.cassandra.cache.IMeasurableMemory; import org.apache.cassandra.config.*; @@ -33,7 +34,8 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.ByteArrayUtil; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; /** * A clustering prefix is the unit of what a {@link ClusteringComparator} can compare. @@ -62,14 +64,21 @@ public enum Kind { // WARNING: the ordering of that enum matters because we use ordinal() in the serialization - EXCL_END_BOUND (0, -1), - INCL_START_BOUND (0, -1), - EXCL_END_INCL_START_BOUNDARY(0, -1), - STATIC_CLUSTERING (1, -1), - CLUSTERING (2, 0), - INCL_END_EXCL_START_BOUNDARY(3, 1), - INCL_END_BOUND (3, 1), - EXCL_START_BOUND (3, 1); + EXCL_END_BOUND ( 0, -1, v -> ByteSource.LT_NEXT_COMPONENT), + INCL_START_BOUND ( 0, -1, v -> ByteSource.LT_NEXT_COMPONENT), + EXCL_END_INCL_START_BOUNDARY( 0, -1, v -> ByteSource.LT_NEXT_COMPONENT), + STATIC_CLUSTERING ( 1, -1, v -> v == Version.LEGACY + ? ByteSource.LT_NEXT_COMPONENT + 1 + : ByteSource.EXCLUDED), + CLUSTERING ( 2, 0, v -> v == Version.LEGACY + ? ByteSource.NEXT_COMPONENT + : ByteSource.TERMINATOR), + INCL_END_EXCL_START_BOUNDARY( 3, 1, v -> ByteSource.GT_NEXT_COMPONENT), + INCL_END_BOUND ( 3, 1, v -> ByteSource.GT_NEXT_COMPONENT), + EXCL_START_BOUND ( 3, 1, v -> ByteSource.GT_NEXT_COMPONENT), + SSTABLE_LOWER_BOUND (-1, -1, v -> ByteSource.LTLT_NEXT_COMPONENT), + SSTABLE_UPPER_BOUND ( 4, 1, v -> ByteSource.GTGT_NEXT_COMPONENT); + private final int comparison; @@ -79,10 +88,13 @@ public enum Kind */ public final int comparedToClustering; - Kind(int comparison, int comparedToClustering) + public final ToIntFunction asByteComparable; + + Kind(int comparison, int comparedToClustering, ToIntFunction asByteComparable) { this.comparison = comparison; this.comparedToClustering = comparedToClustering; + this.asByteComparable = asByteComparable; } /** @@ -197,6 +209,16 @@ public Kind openBoundOfBoundary(boolean reversed) ? (this == INCL_END_EXCL_START_BOUNDARY ? INCL_END_BOUND : EXCL_END_BOUND) : (this == INCL_END_EXCL_START_BOUNDARY ? EXCL_START_BOUND : INCL_START_BOUND); } + + /* + * Returns a terminator value for this clustering type that is suitable for byte comparison. + * Inclusive starts / exclusive ends need a lower value than ByteSource.NEXT_COMPONENT and the clustering byte, + * exclusive starts / inclusive ends -- a higher. + */ + public int asByteComparableValue(Version version) + { + return asByteComparable.applyAsInt(version); + } } default boolean isBottom() @@ -292,6 +314,10 @@ default int dataSize() */ public String toString(TableMetadata metadata); + public ClusteringBound asStartBound(); + + public ClusteringBound asEndBound(); + /* * TODO: we should stop using Clustering for partition keys. Maybe we can add * a few methods to DecoratedKey so we don't have to (note that while using a Clustering @@ -308,6 +334,24 @@ default ByteBuffer serializeAsPartitionKey() values[i] = accessor().toBuffer(get(i)); return CompositeType.build(ByteBufferAccessor.instance, values); } + + /** + * Produce a human-readable representation of the clustering given the list of types. + * Easier to access than metadata for debugging. + */ + public default String clusteringString(List> types) + { + StringBuilder sb = new StringBuilder(); + sb.append(kind()).append('('); + for (int i = 0; i < size(); i++) + { + if (i > 0) + sb.append(", "); + sb.append(types.get(i).getString(get(i), accessor())); + } + return sb.append(')').toString(); + } + /** * The values of this prefix as an array. *

@@ -322,10 +366,11 @@ default ByteBuffer serializeAsPartitionKey() public ByteBuffer[] getBufferArray(); /** - * If the prefix contains byte buffers that can be minimized (see {@link ByteBufferUtil#minimalBufferFor(ByteBuffer)}), - * this will return a copy of the prefix with minimized values, otherwise it returns itself. + * Return the key in a form that can be retained for longer-term use. This means extracting keys stored in shared + * memory (i.e. in memtables) to minimized on-heap versions. + * If the object is already in minimal form, no action will be taken. */ - public ClusteringPrefix minimize(); + public ClusteringPrefix retainable(); public static class Serializer { diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 113a9164b4d0..00d073808cbe 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -20,8 +20,6 @@ import java.io.File; import java.io.IOException; import java.io.PrintStream; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; import java.nio.file.Files; import java.util.*; @@ -29,7 +27,10 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.regex.Pattern; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import javax.management.*; import javax.management.openmbean.*; @@ -49,6 +50,9 @@ import org.apache.cassandra.db.compaction.*; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.DataLimits; +import org.apache.cassandra.db.memtable.Flushing; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.db.memtable.ShardBoundaries; import org.apache.cassandra.db.streaming.CassandraStreamManager; import org.apache.cassandra.db.repair.CassandraTableRepairManager; import org.apache.cassandra.db.view.TableViews; @@ -65,8 +69,10 @@ import org.apache.cassandra.index.transactions.UpdateTransaction; import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.sstable.BloomFilterTracker; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.*; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; @@ -87,7 +93,6 @@ import org.apache.cassandra.utils.*; import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.concurrent.Refs; -import org.apache.cassandra.utils.memory.MemtableAllocator; import org.json.simple.JSONArray; import org.json.simple.JSONObject; @@ -95,8 +100,9 @@ import static org.apache.cassandra.utils.Throwables.maybeFail; import static org.apache.cassandra.utils.Throwables.merge; +import static org.apache.cassandra.utils.Throwables.perform; -public class ColumnFamilyStore implements ColumnFamilyStoreMBean +public class ColumnFamilyStore implements ColumnFamilyStoreMBean, Memtable.Owner { private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class); @@ -134,6 +140,33 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean new NamedThreadFactory("MemtableReclaimMemory"), "internal"); + /** + * Reason for initiating a memtable flush. + */ + public enum FlushReason + { + COMMITLOG_DIRTY, + MEMTABLE_LIMIT, + MEMTABLE_PERIOD_EXPIRED, + INDEX_BUILD_STARTED, + INDEX_BUILD_COMPLETED, + INDEX_REMOVED, + INDEX_TABLE_FLUSH, + VIEW_BUILD_STARTED, + INTERNALLY_FORCED, // explicitly requested flush, necessary for the operation of an internal table + USER_FORCED, // flush explicitly requested by the user (e.g. nodetool flush) + STARTUP, + SHUTDOWN, + SNAPSHOT, + TRUNCATE, + DROP, + STREAMING, + STREAMS_RECEIVED, + REPAIR, + SCHEMA_CHANGE, + UNIT_TESTS; // explicitly requested flush needed for a test + } + private static final String[] COUNTER_NAMES = new String[]{"table", "count", "error", "value"}; private static final String[] COUNTER_DESCS = new String[] { "keyspace.tablename", @@ -167,12 +200,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean private final String oldMBeanName; private volatile boolean valid = true; + private Memtable.Factory memtableFactory; + /** * Memtables and SSTables on disk for this column family. * * We synchronize on the Tracker to ensure isolation when we want to make sure * that the memtable we're acting on doesn't change out from under us. I.e., flush - * syncronizes on it to make sure it can submit on both executors atomically, + * synchronizes on it to make sure it can submit on both executors atomically, * so anyone else who wants to make sure flush doesn't interfere should as well. */ private final Tracker data; @@ -191,7 +226,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean private volatile DefaultValue maxCompactionThreshold; private volatile DefaultValue crcCheckChance; - private final CompactionStrategyManager compactionStrategyManager; + private final CompactionStrategyFactory strategyFactory; + private volatile CompactionStrategyContainer strategyContainer; private final Directories directories; @@ -208,11 +244,21 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean private volatile boolean compactionSpaceCheck = true; + /** The local ranges are used by the {@link DiskBoundaryManager} to create the disk boundaries but can also be + * used independently. They are created lazily and invalidated whenever {@link this#invalidateLocalRangesAndDiskBoundaries()} + * is called. + */ + private volatile SortedLocalRanges localRanges; + @VisibleForTesting final DiskBoundaryManager diskBoundaryManager = new DiskBoundaryManager(); + ShardBoundaries cachedShardBoundaries = null; private volatile boolean neverPurgeTombstones = false; + // BloomFilterTracker is updated from corresponding {@link SSTableReader}s. Metrics are queried via CFS instance. + private final BloomFilterTracker bloomFilterTracker = BloomFilterTracker.createMeterTracker(); + public static void shutdownPostFlushExecutor() throws InterruptedException { postFlushExecutor.shutdown(); @@ -234,7 +280,7 @@ public void reload() // only update these runtime-modifiable settings if they have not been modified. if (!minCompactionThreshold.isModified()) for (ColumnFamilyStore cfs : concatWithIndexes()) - cfs.minCompactionThreshold = new DefaultValue(metadata().params.compaction.minCompactionThreshold()); + cfs.minCompactionThreshold = new DefaultValue<>(metadata().params.compaction.minCompactionThreshold()); if (!maxCompactionThreshold.isModified()) for (ColumnFamilyStore cfs : concatWithIndexes()) cfs.maxCompactionThreshold = new DefaultValue(metadata().params.compaction.maxCompactionThreshold()); @@ -242,50 +288,24 @@ public void reload() for (ColumnFamilyStore cfs : concatWithIndexes()) cfs.crcCheckChance = new DefaultValue(metadata().params.crcCheckChance); - compactionStrategyManager.maybeReload(metadata()); - - scheduleFlush(); + reloadCompactionStrategy(metadata().params.compaction, CompactionStrategyContainer.ReloadReason.METADATA_CHANGE); indexManager.reload(); - // If the CF comparator has changed, we need to change the memtable, - // because the old one still aliases the previous comparator. - if (data.getView().getCurrentMemtable().initialComparator != metadata().comparator) - switchMemtable(); + memtableFactory = metadata().params.memtable.factory; + Memtable currentMemtable = data.getView().getCurrentMemtable(); + if (currentMemtable.shouldSwitch(FlushReason.SCHEMA_CHANGE)) + switchMemtableIfCurrent(currentMemtable, FlushReason.SCHEMA_CHANGE); + else + currentMemtable.metadataUpdated(); } - void scheduleFlush() + /** + * Reload the compaction strategy using the given compaction parameters and reason. + */ + private void reloadCompactionStrategy(CompactionParams compactionParams, CompactionStrategyContainer.ReloadReason reason) { - int period = metadata().params.memtableFlushPeriodInMs; - if (period > 0) - { - logger.trace("scheduling flush in {} ms", period); - WrappedRunnable runnable = new WrappedRunnable() - { - protected void runMayThrow() - { - synchronized (data) - { - Memtable current = data.getView().getCurrentMemtable(); - // if we're not expired, we've been hit by a scheduled flush for an already flushed memtable, so ignore - if (current.isExpired()) - { - if (current.isClean()) - { - // if we're still clean, instead of swapping just reschedule a flush for later - scheduleFlush(); - } - else - { - // we'll be rescheduled by the constructor of the Memtable. - forceFlush(); - } - } - } - } - }; - ScheduledExecutors.scheduledTasks.schedule(runnable, period, TimeUnit.MILLISECONDS); - } + strategyContainer = strategyFactory.reload(strategyContainer, compactionParams, reason); } public static Runnable getBackgroundCompactionTaskSubmitter() @@ -297,9 +317,20 @@ public static Runnable getBackgroundCompactionTaskSubmitter() }; } + @VisibleForTesting + public CompactionStrategyFactory getCompactionFactory() + { + return strategyFactory; + } + + public CompactionParams getCompactionParams() + { + return strategyContainer.getCompactionParams(); + } + public Map getCompactionParameters() { - return compactionStrategyManager.getCompactionParams().asMap(); + return getCompactionParams().asMap(); } public String getCompactionParametersJson() @@ -311,9 +342,7 @@ public void setCompactionParameters(Map options) { try { - CompactionParams compactionParams = CompactionParams.fromMap(options); - compactionParams.validate(); - compactionStrategyManager.setNewLocalCompactionStrategy(compactionParams); + reloadCompactionStrategy(CompactionParams.fromMap(options), CompactionStrategyContainer.ReloadReason.JMX_REQUEST); } catch (Throwable t) { @@ -381,14 +410,15 @@ public ColumnFamilyStore(Keyspace keyspace, fileIndexGenerator.set(generation); sampleReadLatencyNanos = DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS) / 2; additionalWriteLatencyNanos = DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS) / 2; + memtableFactory = metadata.get().params.memtable.factory; logger.info("Initializing {}.{}", keyspace.getName(), name); // Create Memtable only on online Memtable initialMemtable = null; if (DatabaseDescriptor.isDaemonInitialized()) - initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getCurrentPosition()), this); - data = new Tracker(initialMemtable, loadSSTables); + initialMemtable = createMemtable(new AtomicReference<>(CommitLog.instance.getCurrentPosition())); + data = new Tracker(this, initialMemtable, loadSSTables); // Note that this needs to happen before we load the first sstables, or the global sstable tracker will not // be notified on the initial loading. @@ -404,13 +434,10 @@ public ColumnFamilyStore(Keyspace keyspace, } // compaction strategy should be created after the CFS has been prepared - compactionStrategyManager = new CompactionStrategyManager(this); - - if (maxCompactionThreshold.value() <= 0 || minCompactionThreshold.value() <=0) - { - logger.warn("Disabling compaction strategy by setting compaction thresholds to 0 is deprecated, set the compaction option 'enabled' to 'false' instead."); - this.compactionStrategyManager.disable(); - } + this.strategyFactory = new CompactionStrategyFactory(this); + this.strategyContainer = strategyFactory.reload(null, + metadata.get().params.compaction, + CompactionStrategyContainer.ReloadReason.FULL); // create the private ColumnFamilyStores for the secondary column indexes indexManager = new SecondaryIndexManager(this); @@ -419,7 +446,7 @@ public ColumnFamilyStore(Keyspace keyspace, indexManager.addIndex(info, true); } - metric = new TableMetrics(this); + metric = new TableMetrics(this, memtableFactory.createMemtableMetrics(metadata)); if (data.loadsstables) { @@ -510,6 +537,26 @@ public List getDataPaths() throws IOException return dataPaths; } + public boolean writesShouldSkipCommitLog() + { + return memtableFactory.writesShouldSkipCommitLog(); + } + + public boolean memtableWritesAreDurable() + { + return memtableFactory.writesAreDurable(); + } + + public boolean streamToMemtable() + { + return memtableFactory.streamToMemtable(); + } + + public boolean streamFromMemtable() + { + return memtableFactory.streamFromMemtable(); + } + public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, boolean isTransient, int sstableLevel, SerializationHeader header, LifecycleNewTracker lifecycleNewTracker) { MetadataCollector collector = new MetadataCollector(metadata().comparator).sstableLevel(sstableLevel); @@ -518,12 +565,12 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long k public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, UUID pendingRepair, boolean isTransient, MetadataCollector metadataCollector, SerializationHeader header, LifecycleNewTracker lifecycleNewTracker) { - return getCompactionStrategyManager().createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadataCollector, header, indexManager.listIndexes(), lifecycleNewTracker); + return getCompactionStrategy().createSSTableMultiWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadataCollector, header, indexManager.listIndexGroups(), lifecycleNewTracker); } public boolean supportsEarlyOpen() { - return compactionStrategyManager.supportsEarlyOpen(); + return strategyContainer.supportsEarlyOpen(); } /** call when dropping or renaming a CF. Performs mbean housekeeping and invalidates CFS to other operations */ @@ -551,7 +598,7 @@ public void invalidate(boolean expectMBean) } } - compactionStrategyManager.shutdown(); + strategyContainer.shutdown(); SystemKeyspace.removeTruncationRecord(metadata.id); data.dropSSTables(); @@ -774,20 +821,6 @@ public static void rebuildSecondaryIndex(String ksName, String cfName, String... cfs.indexManager.rebuildIndexesBlocking(Sets.newHashSet(Arrays.asList(idxNames))); } - public AbstractCompactionStrategy createCompactionStrategyInstance(CompactionParams compactionParams) - { - try - { - Constructor constructor = - compactionParams.klass().getConstructor(ColumnFamilyStore.class, Map.class); - return constructor.newInstance(this, compactionParams.options()); - } - catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException e) - { - throw new RuntimeException(e); - } - } - @Deprecated public String getColumnFamilyName() { @@ -799,6 +832,11 @@ public String getTableName() return name; } + public String getKeyspaceName() + { + return keyspace.getName(); + } + public Descriptor newSSTableDescriptor(File directory) { return newSSTableDescriptor(directory, SSTableFormat.Type.current().info.getLatestVersion(), SSTableFormat.Type.current()); @@ -824,12 +862,12 @@ public Descriptor newSSTableDescriptor(File directory, Version version, SSTableF * * @param memtable */ - public ListenableFuture switchMemtableIfCurrent(Memtable memtable) + public ListenableFuture switchMemtableIfCurrent(Memtable memtable, FlushReason reason) { synchronized (data) { if (data.getView().getCurrentMemtable() == memtable) - return switchMemtable(); + return switchMemtable(reason); } logger.debug("Memtable is no longer current, returning future that completes when current flushing operation completes"); return waitForFlushes(); @@ -842,11 +880,12 @@ public ListenableFuture switchMemtableIfCurrent(Memtable memt * not complete until the Memtable (and all prior Memtables) have been successfully flushed, and the CL * marked clean up to the position owned by the Memtable. */ - public ListenableFuture switchMemtable() + @VisibleForTesting + public ListenableFuture switchMemtable(FlushReason reason) { synchronized (data) { - logFlush(); + logFlush(reason); Flush flush = new Flush(false); flushExecutor.execute(flush); postFlushExecutor.execute(flush.postFlushTask); @@ -855,33 +894,19 @@ public ListenableFuture switchMemtable() } // print out size of all memtables we're enqueuing - private void logFlush() + private void logFlush(FlushReason reason) { // reclaiming includes that which we are GC-ing; - float onHeapRatio = 0, offHeapRatio = 0; - long onHeapTotal = 0, offHeapTotal = 0; - Memtable memtable = getTracker().getView().getCurrentMemtable(); - onHeapRatio += memtable.getAllocator().onHeap().ownershipRatio(); - offHeapRatio += memtable.getAllocator().offHeap().ownershipRatio(); - onHeapTotal += memtable.getAllocator().onHeap().owns(); - offHeapTotal += memtable.getAllocator().offHeap().owns(); + Memtable.MemoryUsage usage = Memtable.newMemoryUsage(); + getTracker().getView().getCurrentMemtable().addMemoryUsageTo(usage); for (ColumnFamilyStore indexCfs : indexManager.getAllIndexColumnFamilyStores()) - { - MemtableAllocator allocator = indexCfs.getTracker().getView().getCurrentMemtable().getAllocator(); - onHeapRatio += allocator.onHeap().ownershipRatio(); - offHeapRatio += allocator.offHeap().ownershipRatio(); - onHeapTotal += allocator.onHeap().owns(); - offHeapTotal += allocator.offHeap().owns(); - } + indexCfs.getTracker().getView().getCurrentMemtable().addMemoryUsageTo(usage); - logger.info("Enqueuing flush of {}: {}", + logger.info("Enqueuing flush of {} ({}): {}", name, - String.format("%s (%.0f%%) on-heap, %s (%.0f%%) off-heap", - FBUtilities.prettyPrintMemory(onHeapTotal), - onHeapRatio * 100, - FBUtilities.prettyPrintMemory(offHeapTotal), - offHeapRatio * 100)); + reason, + usage); } @@ -891,14 +916,14 @@ private void logFlush() * @return a Future yielding the commit log position that can be guaranteed to have been successfully written * to sstables for this table once the future completes */ - public ListenableFuture forceFlush() + public ListenableFuture forceFlush(FlushReason reason) { synchronized (data) { Memtable current = data.getView().getCurrentMemtable(); for (ColumnFamilyStore cfs : concatWithIndexes()) if (!cfs.data.getView().getCurrentMemtable().isClean()) - return switchMemtableIfCurrent(current); + return flushMemtable(current, reason); return waitForFlushes(); } } @@ -916,10 +941,18 @@ public ListenableFuture forceFlush(CommitLogPosition flushIfDirtyBefore) // and this does not vary between a table and its table-backed indexes Memtable current = data.getView().getCurrentMemtable(); if (current.mayContainDataBefore(flushIfDirtyBefore)) - return switchMemtableIfCurrent(current); + return flushMemtable(current, FlushReason.COMMITLOG_DIRTY); return waitForFlushes(); } + private ListenableFuture flushMemtable(Memtable current, FlushReason reason) + { + if (current.shouldSwitch(reason)) + return switchMemtableIfCurrent(current, reason); + else + return waitForFlushes(); + } + /** * @return a Future yielding the commit log position that can be guaranteed to have been successfully written * to sstables for this table once the future completes @@ -929,17 +962,14 @@ private ListenableFuture waitForFlushes() // we grab the current memtable; once any preceding memtables have flushed, we know its // commitLogLowerBound has been set (as this it is set with the upper bound of the preceding memtable) final Memtable current = data.getView().getCurrentMemtable(); - ListenableFutureTask task = ListenableFutureTask.create(() -> { - logger.debug("forceFlush requested but everything is clean in {}", name); - return current.getCommitLogLowerBound(); - }); + ListenableFutureTask task = ListenableFutureTask.create(current::getCommitLogLowerBound); postFlushExecutor.execute(task); return task; } - public CommitLogPosition forceBlockingFlush() + public CommitLogPosition forceBlockingFlush(FlushReason reason) { - return FBUtilities.waitOnFuture(forceFlush()); + return FBUtilities.waitOnFuture(forceFlush(reason)); } /** @@ -949,12 +979,12 @@ public CommitLogPosition forceBlockingFlush() private final class PostFlush implements Callable { final CountDownLatch latch = new CountDownLatch(1); - final List memtables; + final Memtable mainMemtable; volatile Throwable flushFailure = null; - private PostFlush(List memtables) + private PostFlush(Memtable mainMemtable) { - this.memtables = memtables; + this.mainMemtable = mainMemtable; } public CommitLogPosition call() @@ -972,11 +1002,10 @@ public CommitLogPosition call() CommitLogPosition commitLogUpperBound = CommitLogPosition.NONE; // If a flush errored out but the error was ignored, make sure we don't discard the commit log. - if (flushFailure == null && !memtables.isEmpty()) + if (flushFailure == null && mainMemtable != null) { - Memtable memtable = memtables.get(0); - commitLogUpperBound = memtable.getCommitLogUpperBound(); - CommitLog.instance.discardCompletedSegments(metadata.id, memtable.getCommitLogLowerBound(), commitLogUpperBound); + commitLogUpperBound = mainMemtable.getFinalCommitLogUpperBound(); + CommitLog.instance.discardCompletedSegments(metadata.id, mainMemtable.getCommitLogLowerBound(), commitLogUpperBound); } metric.pendingFlushes.dec(); @@ -999,7 +1028,7 @@ public CommitLogPosition call() private final class Flush implements Runnable { final OpOrder.Barrier writeBarrier; - final List memtables = new ArrayList<>(); + final Map memtables; final ListenableFutureTask postFlushTask; final PostFlush postFlush; final boolean truncate; @@ -1023,6 +1052,8 @@ private Flush(boolean truncate) */ writeBarrier = Keyspace.writeOrder.newBarrier(); + memtables = new LinkedHashMap<>(); + // submit flushes for the memtable for any indexed sub-cfses, and our own AtomicReference commitLogUpperBound = new AtomicReference<>(); for (ColumnFamilyStore cfs : concatWithIndexes()) @@ -1030,10 +1061,10 @@ private Flush(boolean truncate) // switch all memtables, regardless of their dirty status, setting the barrier // so that we can reach a coordinated decision about cleanliness once they // are no longer possible to be modified - Memtable newMemtable = new Memtable(commitLogUpperBound, cfs); + Memtable newMemtable = cfs.createMemtable(commitLogUpperBound); Memtable oldMemtable = cfs.data.switchMemtable(truncate, newMemtable); - oldMemtable.setDiscarding(writeBarrier, commitLogUpperBound); - memtables.add(oldMemtable); + oldMemtable.switchOut(writeBarrier, commitLogUpperBound); + memtables.put(cfs, oldMemtable); } // we then ensure an atomic decision is made about the upper bound of the continuous range of commit log @@ -1044,7 +1075,7 @@ private Flush(boolean truncate) // since this happens after wiring up the commitLogUpperBound, we also know all operations with earlier // commit log segment position have also completed, i.e. the memtables are done and ready to flush writeBarrier.issue(); - postFlush = new PostFlush(memtables); + postFlush = new PostFlush(Iterables.get(memtables.values(), 0, null)); postFlushTask = ListenableFutureTask.create(postFlush); } @@ -1064,17 +1095,20 @@ public void run() logger.trace("Flush task for task {}@{} waited {} ms at the barrier", hashCode(), name, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); // mark all memtables as flushing, removing them from the live memtable list - for (Memtable memtable : memtables) - memtable.cfs.data.markFlushing(memtable); + for (Map.Entry entry : memtables.entrySet()) + entry.getKey().data.markFlushing(entry.getValue()); metric.memtableSwitchCount.inc(); try { + boolean first = true; // Flush "data" memtable with non-cf 2i first; - flushMemtable(memtables.get(0), true); - for (int i = 1; i < memtables.size(); i++) - flushMemtable(memtables.get(i), false); + for (Map.Entry entry : memtables.entrySet()) + { + flushMemtable(entry.getKey(), entry.getValue(), first); + first = false; + } } catch (Throwable t) { @@ -1092,14 +1126,14 @@ public void run() logger.trace("Flush task task {}@{} finished", hashCode(), name); } - public Collection flushMemtable(Memtable memtable, boolean flushNonCf2i) + public Collection flushMemtable(ColumnFamilyStore cfs, Memtable memtable, boolean flushNonCf2i) { if (logger.isTraceEnabled()) logger.trace("Flush task task {}@{} flushing memtable {}", hashCode(), name, memtable); if (memtable.isClean() || truncate) { - memtable.cfs.replaceFlushed(memtable, Collections.emptyList()); + cfs.replaceFlushed(memtable, Collections.emptyList()); reclaim(memtable); return Collections.emptyList(); } @@ -1111,13 +1145,13 @@ public Collection flushMemtable(Memtable memtable, boolean flushN List sstables = new ArrayList<>(); try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.FLUSH)) { - List flushRunnables = null; + List flushRunnables = null; List flushResults = null; try { // flush the memtable - flushRunnables = memtable.flushRunnables(txn); + flushRunnables = Flushing.flushRunnables(cfs, memtable, txn); ExecutorService[] executors = perDiskflushExecutors.getExecutorsFor(keyspace.getName(), name); for (int i = 0; i < flushRunnables.size(); i++) @@ -1132,12 +1166,25 @@ public Collection flushMemtable(Memtable memtable, boolean flushN if (flushNonCf2i) indexManager.flushAllNonCFSBackedIndexesBlocking(); + // It may be worthwhile to add an early abort mechanism here if one of the futures throws. + // In such a case this code will run the other threads to completion and only then abort the operation. flushResults = Lists.newArrayList(FBUtilities.waitOnFutures(futures)); } catch (Throwable t) { - t = memtable.abortRunnables(flushRunnables, t); + logger.error("Flushing {} failed with error", memtable.toString(), t); + t = Flushing.abortRunnables(flushRunnables, t); + + // wait for any flush runnables that were submitted (after aborting they should complete immediately) + // this ensures that the writers are aborted by FlushRunnable.writeSortedContents(), in the worst + // case we'll repeat the same exception twice if the initial exception was thrown whilst waiting + // on a future + t = perform(t, () -> FBUtilities.waitOnFutures(futures)); + + //finally abort the transaction t = txn.abort(t); + + // and re-throw throw Throwables.propagate(t); } @@ -1148,7 +1195,7 @@ public Collection flushMemtable(Memtable memtable, boolean flushN { @SuppressWarnings("resource") SSTableMultiWriter writer = writerIterator.next(); - if (writer.getFilePointer() > 0) + if (writer.getBytesWritten() > 0) { writer.setOpenResult(true).prepareToCommit(); } @@ -1191,9 +1238,9 @@ public Collection flushMemtable(Memtable memtable, boolean flushN } } } - memtable.cfs.replaceFlushed(memtable, sstables); + cfs.replaceFlushed(memtable, sstables); reclaim(memtable); - memtable.cfs.compactionStrategyManager.compactionLogger.flush(sstables); + cfs.strategyFactory.getCompactionLogger().flush(sstables); logger.debug("Flushed to {} ({} sstables, {}), biggest {}, smallest {}", sstables, sstables.size(), @@ -1213,12 +1260,17 @@ private void reclaim(final Memtable memtable) public void runMayThrow() { readBarrier.await(); - memtable.setDiscarded(); + memtable.discard(); } }, reclaimExecutor); } } + public Memtable createMemtable(AtomicReference commitLogUpperBound) + { + return memtableFactory.create(commitLogUpperBound, metadata, this); + } + // atomically set the upper bound for the commit log private static void setCommitLogUpperBound(AtomicReference commitLogUpperBound) { @@ -1236,108 +1288,53 @@ private static void setCommitLogUpperBound(AtomicReference co } } - /** - * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately - * queues it for flushing. If the memtable selected is flushed before this completes, no work is done. - */ - public static CompletableFuture flushLargestMemtable() + public ListenableFuture signalFlushRequired(Memtable memtable, FlushReason reason) { - float largestRatio = 0f; - Memtable largest = null; - float liveOnHeap = 0, liveOffHeap = 0; - for (ColumnFamilyStore cfs : ColumnFamilyStore.all()) - { - // we take a reference to the current main memtable for the CF prior to snapping its ownership ratios - // to ensure we have some ordering guarantee for performing the switchMemtableIf(), i.e. we will only - // swap if the memtables we are measuring here haven't already been swapped by the time we try to swap them - Memtable current = cfs.getTracker().getView().getCurrentMemtable(); - - // find the total ownership ratio for the memtable and all SecondaryIndexes owned by this CF, - // both on- and off-heap, and select the largest of the two ratios to weight this CF - float onHeap = 0f, offHeap = 0f; - onHeap += current.getAllocator().onHeap().ownershipRatio(); - offHeap += current.getAllocator().offHeap().ownershipRatio(); - - for (ColumnFamilyStore indexCfs : cfs.indexManager.getAllIndexColumnFamilyStores()) - { - MemtableAllocator allocator = indexCfs.getTracker().getView().getCurrentMemtable().getAllocator(); - onHeap += allocator.onHeap().ownershipRatio(); - offHeap += allocator.offHeap().ownershipRatio(); - } - - float ratio = Math.max(onHeap, offHeap); - if (ratio > largestRatio) - { - largest = current; - largestRatio = ratio; - } - - liveOnHeap += onHeap; - liveOffHeap += offHeap; - } - - CompletableFuture returnFuture = new CompletableFuture<>(); - - if (largest != null) - { - float usedOnHeap = Memtable.MEMORY_POOL.onHeap.usedRatio(); - float usedOffHeap = Memtable.MEMORY_POOL.offHeap.usedRatio(); - float flushingOnHeap = Memtable.MEMORY_POOL.onHeap.reclaimingRatio(); - float flushingOffHeap = Memtable.MEMORY_POOL.offHeap.reclaimingRatio(); - float thisOnHeap = largest.getAllocator().onHeap().ownershipRatio(); - float thisOffHeap = largest.getAllocator().offHeap().ownershipRatio(); - logger.debug("Flushing largest {} to free up room. Used total: {}, live: {}, flushing: {}, this: {}", - largest.cfs, ratio(usedOnHeap, usedOffHeap), ratio(liveOnHeap, liveOffHeap), - ratio(flushingOnHeap, flushingOffHeap), ratio(thisOnHeap, thisOffHeap)); - - ListenableFuture flushFuture = largest.cfs.switchMemtableIfCurrent(largest); - flushFuture.addListener(() -> { - try - { - flushFuture.get(); - returnFuture.complete(true); - } - catch (Throwable t) - { - returnFuture.completeExceptionally(t); - } - }, MoreExecutors.directExecutor()); - } - else - { - logger.debug("Flushing of largest memtable, not done, no memtable found"); + return switchMemtableIfCurrent(memtable, reason); + } - returnFuture.complete(false); - } + @Override + public Memtable getCurrentMemtable() + { + return data.getView().getCurrentMemtable(); + } - return returnFuture; + public static Iterable activeMemtables() + { + return Iterables.transform(ColumnFamilyStore.all(), + cfs -> cfs.getTracker().getView().getCurrentMemtable()); } - private static String ratio(float onHeap, float offHeap) + public Iterable getIndexMemtables() { - return String.format("%.2f/%.2f", onHeap, offHeap); + return Iterables.transform(indexManager.getAllIndexColumnFamilyStores(), + cfs -> cfs.getTracker().getView().getCurrentMemtable()); } /** * Insert/Update the column family for this key. * Caller is responsible for acquiring Keyspace.switchLock - * param @ lock - lock that needs to be used. - * param @ key - key for update/insert - * param @ columnFamily - columnFamily changes + * @param update to be applied + * @param context write context for current update + * @param updateIndexes whether secondary indexes should be updated */ - public void apply(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup, CommitLogPosition commitLogPosition) - + @SuppressWarnings("resource") // opGroup + public void apply(PartitionUpdate update, CassandraWriteContext context, boolean updateIndexes) { long start = System.nanoTime(); + OpOrder.Group opGroup = context.getGroup(); + CommitLogPosition commitLogPosition = context.getPosition(); try { Memtable mt = data.getMemtableFor(opGroup, commitLogPosition); + UpdateTransaction indexer = newUpdateTransaction(update, context, updateIndexes, mt); long timeDelta = mt.put(update, indexer, opGroup); DecoratedKey key = update.partitionKey(); invalidateCachedPartition(key); metric.topWritePartitionFrequency.addSample(key.getKey(), 1); if (metric.topWritePartitionSize.isEnabled()) // dont compute datasize if not needed metric.topWritePartitionSize.addSample(key.getKey(), update.dataSize()); + metric.bytesInserted.inc(update.dataSize()); StorageHook.instance.reportWrite(metadata.id, update); metric.writeLatency.addNano(System.nanoTime() - start); // CASSANDRA-11117 - certain resolution paths on memtable put can result in very @@ -1356,6 +1353,33 @@ public void apply(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Gro } } + private UpdateTransaction newUpdateTransaction(PartitionUpdate update, CassandraWriteContext context, boolean updateIndexes, Memtable memtable) + { + return updateIndexes + ? indexManager.newUpdateTransaction(update, context, FBUtilities.nowInSeconds(), memtable) + : UpdateTransaction.NO_OP; + } + + public ShardBoundaries localRangeSplits(int shardCount) + { + if (shardCount == 1 || !getPartitioner().splitter().isPresent() || SchemaConstants.isLocalSystemKeyspace(keyspace.getName())) + return ShardBoundaries.NONE; + + ShardBoundaries shardBoundaries = cachedShardBoundaries; + if (shardBoundaries == null || + shardBoundaries.shardCount() != shardCount || + shardBoundaries.ringVersion != StorageService.instance.getTokenMetadata().getRingVersion()) + { + SortedLocalRanges localRanges = getLocalRanges(); + List positions = localRanges.split(shardCount); + shardBoundaries = new ShardBoundaries(positions.subList(0, positions.size() - 1), + localRanges.getRingVersion()); + cachedShardBoundaries = shardBoundaries; + logger.info("Memtable shard boundaries for {}.{}: {}", keyspace.getName(), getTableName(), positions); + } + return shardBoundaries; + } + /** * @param sstables * @return sstables whose key range overlaps with that of the given sstables, not including itself. @@ -1522,7 +1546,7 @@ public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boole { // skip snapshot creation during scrub, SEE JIRA 5891 if(!disableSnapshot) - snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis()); + snapshotWithoutMemtable("pre-scrub-" + System.currentTimeMillis()); try { @@ -1609,7 +1633,7 @@ public Tracker getTracker() public Set getLiveSSTables() { - return data.getView().liveSSTables(); + return data.getLiveSSTables(); } public Iterable getSSTables(SSTableSet sstableSet) @@ -1617,9 +1641,19 @@ public Iterable getSSTables(SSTableSet sstableSet) return data.getView().select(sstableSet); } - public Iterable getUncompactingSSTables() + public Iterable getNoncompactingSSTables() + { + return data.getNoncompacting(); + } + + public Iterable getNoncompactingSSTables(Iterable candidates) { - return data.getUncompacting(); + return data.getNoncompacting(candidates); + } + + public Set getCompactingSSTables() + { + return data.getCompacting(); } public Map getPendingRepairStats() @@ -1645,29 +1679,6 @@ public Map getPendingRepairStats() return stats; } - /** - * promotes (or demotes) data attached to an incremental repair session that has either completed successfully, - * or failed - * - * @return session ids whose data could not be released - */ - public CleanupSummary releaseRepairData(Collection sessions, boolean force) - { - if (force) - { - Predicate predicate = sst -> { - UUID session = sst.getPendingRepair(); - return session != null && sessions.contains(session); - }; - return runWithCompactionsDisabled(() -> compactionStrategyManager.releaseRepairData(sessions), - predicate, false, true, true); - } - else - { - return compactionStrategyManager.releaseRepairData(sessions); - } - } - public boolean isFilterFullyCoveredBy(ClusteringIndexFilter filter, DataLimits limits, CachedPartition cached, @@ -1752,7 +1763,7 @@ public List getSSTablesForKey(String key, boolean hexFormat) for (SSTableReader sstr : select(View.select(SSTableSet.LIVE, dk)).sstables) { // check if the key actually exists in this sstable, without updating cache and stats - if (sstr.getPosition(dk, SSTableReader.Operator.EQ, false) != null) + if (sstr.checkEntryExists(dk, SSTableReader.Operator.EQ, false)) files.add(sstr.getFilename()); } return files; @@ -1824,15 +1835,15 @@ public ClusteringComparator getComparator() return metadata().comparator; } - public void snapshotWithoutFlush(String snapshotName) + public void snapshotWithoutMemtable(String snapshotName) { - snapshotWithoutFlush(snapshotName, null, false, null); + snapshotWithoutMemtable(snapshotName, null, false, null); } /** * @param ephemeral If this flag is set to true, the snapshot will be cleaned during next startup */ - public Set snapshotWithoutFlush(String snapshotName, Predicate predicate, boolean ephemeral, RateLimiter rateLimiter) + public Set snapshotWithoutMemtable(String snapshotName, Predicate predicate, boolean ephemeral, RateLimiter rateLimiter) { if (rateLimiter == null) rateLimiter = DatabaseDescriptor.getSnapshotRateLimiter(); @@ -1962,7 +1973,7 @@ public Refs getSnapshotSSTableReaders(String tag) throws IOExcept if (logger.isTraceEnabled()) logger.trace("using snapshot sstable {}", entries.getKey()); // open offline so we don't modify components or track hotness. - sstable = SSTableReader.open(entries.getKey(), entries.getValue(), metadata, true, true); + sstable = entries.getKey().getFormat().getReaderFactory().open(entries.getKey(), entries.getValue(), metadata, true, true); refs.tryRef(sstable); // release the self ref as we never add the snapshot sstable to DataTracker where it is otherwise released sstable.selfRef().release(); @@ -1997,36 +2008,43 @@ public Set snapshot(String snapshotName) * Take a snap shot of this columnfamily store. * * @param snapshotName the name of the associated with the snapshot - * @param skipFlush Skip blocking flush of memtable + * @param skipMemtable Skip flushing the memtable * @param rateLimiter Rate limiter for hardlinks-per-second */ - public Set snapshot(String snapshotName, boolean skipFlush, RateLimiter rateLimiter) + public Set snapshot(String snapshotName, boolean skipMemtable, RateLimiter rateLimiter) { - return snapshot(snapshotName, null, false, skipFlush, rateLimiter); + return snapshot(snapshotName, null, false, skipMemtable, rateLimiter); } /** * @param ephemeral If this flag is set to true, the snapshot will be cleaned up during next startup - * @param skipFlush Skip blocking flush of memtable + * @param skipMemtable Skip flushing the memtable */ - public Set snapshot(String snapshotName, Predicate predicate, boolean ephemeral, boolean skipFlush) + public Set snapshot(String snapshotName, Predicate predicate, boolean ephemeral, boolean skipMemtable) { - return snapshot(snapshotName, predicate, ephemeral, skipFlush, null); + return snapshot(snapshotName, predicate, ephemeral, skipMemtable, null); } /** * @param ephemeral If this flag is set to true, the snapshot will be cleaned up during next startup - * @param skipFlush Skip blocking flush of memtable + * @param skipMemtable Skip flushing the memtable * @param rateLimiter Rate limiter for hardlinks-per-second */ - public Set snapshot(String snapshotName, Predicate predicate, boolean ephemeral, boolean skipFlush, RateLimiter rateLimiter) + public Set snapshot(String snapshotName, Predicate predicate, boolean ephemeral, boolean skipMemtable, RateLimiter rateLimiter) { - if (!skipFlush) + if (!skipMemtable) { - forceBlockingFlush(); + Memtable current = getTracker().getView().getCurrentMemtable(); + if (!current.isClean()) + { + if (current.shouldSwitch(FlushReason.SNAPSHOT)) + FBUtilities.waitOnFuture(switchMemtableIfCurrent(current, FlushReason.SNAPSHOT)); + else + current.performSnapshot(snapshotName); + } } - return snapshotWithoutFlush(snapshotName, predicate, ephemeral, rateLimiter); + return snapshotWithoutMemtable(snapshotName, predicate, ephemeral, rateLimiter); } public boolean snapshotExists(String snapshotName) @@ -2205,6 +2223,108 @@ public long estimatedKeysForRange(Range range) } } + public void writeAndAddMemtableRanges(UUID repairSessionID, + Supplier>> rangesSupplier, + Refs placeIntoRefs) + { + @SuppressWarnings("resource") // closed by finish or on exception + SSTableMultiWriter memtableContent = writeMemtableRanges(rangesSupplier, repairSessionID); + if (memtableContent != null) + { + try + { + Collection sstables = memtableContent.finish(true); + try (Refs sstableReferences = Refs.ref(sstables)) + { + // This moves all references to placeIntoRefs, clearing sstableReferences + placeIntoRefs.addAll(sstableReferences); + } + + // Release the reference any written sstables start with. + for (SSTableReader rdr : sstables) + { + rdr.selfRef().release(); + logger.info("Memtable ranges (keys {} size {}) written in {}", + rdr.estimatedKeys(), + rdr.getDataChannel().size(), + rdr); + } + } + catch (Throwable t) + { + memtableContent.close(); + Throwables.propagate(t); + } + } + } + + private SSTableMultiWriter writeMemtableRanges(Supplier>> rangesSupplier, + UUID repairSessionID) + { + if (!streamFromMemtable()) + return null; + + Collection> ranges = rangesSupplier.get(); + Memtable current = getTracker().getView().getCurrentMemtable(); + if (current.isClean()) + return null; + + List> dataSets = new ArrayList<>(ranges.size()); + long keys = 0; + for (Range range : ranges) + { + Memtable.FlushCollection dataSet = current.getFlushSet(range.left, range.right); + dataSets.add(dataSet); + keys += dataSet.partitionCount(); + } + if (keys == 0) + return null; + + // TODO: Can we write directly to stream, skipping disk? + Memtable.FlushCollection firstDataSet = dataSets.get(0); + SSTableMultiWriter writer = createSSTableMultiWriter(newSSTableDescriptor(directories.getDirectoryForNewSSTables()), + keys, + 0, + repairSessionID, + false, + 0, + new SerializationHeader(true, + firstDataSet.metadata(), + firstDataSet.columns(), + firstDataSet.encodingStats()), + DO_NOT_TRACK); + try + { + for (Memtable.FlushCollection dataSet : dataSets) + new Flushing.FlushRunnable(dataSet, writer, metric, false).call(); // executes on this thread + + return writer; + } + catch (Error | RuntimeException t) + { + writer.abort(t); + throw t; + } + } + + private static final LifecycleNewTracker DO_NOT_TRACK = new LifecycleNewTracker() + { + public void trackNew(SSTable table) + { + // not tracking + } + + public void untrackNew(SSTable table) + { + // not tracking + } + + public OperationType opType() + { + return OperationType.FLUSH; + } + }; + /** * For testing. No effort is made to clear historical or even the current memtables, nor for * thread safety. All we do is wipe the sstable containers clean, while leaving the actual @@ -2216,7 +2336,8 @@ public void clearUnsafe() for (final ColumnFamilyStore cfs : concatWithIndexes()) { cfs.runWithCompactionsDisabled((Callable) () -> { - cfs.data.reset(new Memtable(new AtomicReference<>(CommitLogPosition.NONE), cfs)); + cfs.data.reset(memtableFactory.create(new AtomicReference<>(CommitLogPosition.NONE), cfs.metadata, cfs)); + cfs.reloadCompactionStrategy(metadata().params.compaction, CompactionStrategyContainer.ReloadReason.FULL); return null; }, true, false); } @@ -2226,6 +2347,14 @@ public void clearUnsafe() * Truncate deletes the entire column family's data with no expensive tombstone creation */ public void truncateBlocking() + { + truncateBlocking(DatabaseDescriptor.isAutoSnapshot()); + } + + /** + * Truncate deletes the entire column family's data with no expensive tombstone creation + */ + public void truncateBlocking(boolean snapshot) { // We have two goals here: // - truncate should delete everything written before truncate was invoked @@ -2246,23 +2375,19 @@ public void truncateBlocking() final long truncatedAt; final CommitLogPosition replayAfter; - if (keyspace.getMetadata().params.durableWrites || DatabaseDescriptor.isAutoSnapshot()) + if ((keyspace.getMetadata().params.durableWrites && !memtableWritesAreDurable()) // need to clear dirty regions + || snapshot) // need sstable for snapshot { - replayAfter = forceBlockingFlush(); - viewManager.forceBlockingFlush(); + replayAfter = forceBlockingFlush(FlushReason.TRUNCATE); + viewManager.forceBlockingFlush(FlushReason.TRUNCATE); } else { // just nuke the memtable data w/o writing to disk first - viewManager.dumpMemtables(); - try - { - replayAfter = dumpMemtable().get(); - } - catch (Exception e) - { - throw new RuntimeException(e); - } + // note: this does not wait for the switch to complete, but because the post-flush processing is serial, + // the call below does. + viewManager.dumpMemtables(FlushReason.TRUNCATE); + replayAfter = FBUtilities.waitOnFuture(dumpMemtable(FlushReason.TRUNCATE)); } long now = System.currentTimeMillis(); @@ -2284,13 +2409,13 @@ public void run() "Stopping parent sessions {} due to truncation of tableId="+metadata.id); data.notifyTruncated(truncatedAt); - if (DatabaseDescriptor.isAutoSnapshot()) - snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(name, SNAPSHOT_TRUNCATE_PREFIX)); + if (snapshot) + snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(name, SNAPSHOT_TRUNCATE_PREFIX)); - discardSSTables(truncatedAt); + discardSSTables(truncatedAt); - indexManager.truncateAllIndexesBlocking(truncatedAt); - viewManager.truncateBlocking(replayAfter, truncatedAt); + indexManager.truncateAllIndexesBlocking(truncatedAt); + viewManager.truncateBlocking(replayAfter, truncatedAt); SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter); logger.trace("cleaning out row cache"); @@ -2299,17 +2424,16 @@ public void run() } }; - runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true); + runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true, AbstractTableOperation.StopTrigger.TRUNCATE); viewManager.build(); - logger.info("Truncate of {}.{} is complete", keyspace.getName(), name); } /** * Drops current memtable without flushing to disk. This should only be called when truncating a column family which is not durable. */ - public Future dumpMemtable() + public Future dumpMemtable(FlushReason reason) { synchronized (data) { @@ -2320,11 +2444,29 @@ public Future dumpMemtable() } } + public void unloadCf() + { + if (keyspace.getMetadata().params.durableWrites && !memtableWritesAreDurable()) // need to clear dirty regions + forceBlockingFlush(ColumnFamilyStore.FlushReason.DROP); + else + FBUtilities.waitOnFuture(dumpMemtable(ColumnFamilyStore.FlushReason.DROP)); + } + public V runWithCompactionsDisabled(Callable callable, boolean interruptValidation, boolean interruptViews) { return runWithCompactionsDisabled(callable, (sstable) -> true, interruptValidation, interruptViews, true); } + public V runWithCompactionsDisabled(Callable callable, boolean interruptValidation, boolean interruptViews, AbstractTableOperation.StopTrigger trigger) + { + return runWithCompactionsDisabled(callable, (sstable) -> true, interruptValidation, interruptViews, true, trigger); + } + + public V runWithCompactionsDisabled(Callable callable, Predicate sstablesPredicate, boolean interruptValidation, boolean interruptViews, boolean interruptIndexes) + { + return runWithCompactionsDisabled(callable, sstablesPredicate, interruptValidation, interruptViews, interruptIndexes, AbstractTableOperation.StopTrigger.NONE); + } + /** * Runs callable with compactions paused and compactions including sstables matching sstablePredicate stopped * @@ -2335,7 +2477,7 @@ public V runWithCompactionsDisabled(Callable callable, boolean interruptV * @param interruptIndexes if we should interrupt compactions on indexes. NOTE: if you set this to true your sstablePredicate * must be able to handle LocalPartitioner sstables! */ - public V runWithCompactionsDisabled(Callable callable, Predicate sstablesPredicate, boolean interruptValidation, boolean interruptViews, boolean interruptIndexes) + public V runWithCompactionsDisabled(Callable callable, Predicate sstablesPredicate, boolean interruptValidation, boolean interruptViews, boolean interruptIndexes, AbstractTableOperation.StopTrigger trigger) { // synchronize so that concurrent invocations don't re-enable compactions partway through unexpectedly, // and so we only run one major compaction at a time @@ -2354,7 +2496,7 @@ public V runWithCompactionsDisabled(Callable callable, Predicate maybeFail(resumeAll(null, toPause)); @@ -2407,7 +2549,7 @@ private static Throwable resumeAll(Throwable accumulate, Iterable callable = () -> { assert data.getCompacting().isEmpty() : data.getCompacting(); - Iterable sstables = getLiveSSTables(); - sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables); + Iterable sstables = Iterables.filter(getLiveSSTables(), sstable -> !sstable.isMarkedSuspect()); LifecycleTransaction modifier = data.tryModify(sstables, operationType); assert modifier != null: "something marked things compacting while compactions are disabled"; return modifier; @@ -2445,7 +2586,12 @@ public void disableAutoCompaction() { // we don't use CompactionStrategy.pause since we don't want users flipping that on and off // during runWithCompactionsDisabled - compactionStrategyManager.disable(); + strategyContainer.disable(); + } + + public boolean compactionShouldBeEnabled() + { + return strategyContainer.getCompactionParams().isEnabled(); } public void enableAutoCompaction() @@ -2455,34 +2601,79 @@ public void enableAutoCompaction() /** * used for tests - to be able to check things after a minor compaction - * @param waitForFutures if we should block until autocompaction is done + * @param waitForFuture if we should block until autocompaction is done */ @VisibleForTesting - public void enableAutoCompaction(boolean waitForFutures) + public void enableAutoCompaction(boolean waitForFuture) { - compactionStrategyManager.enable(); - List> futures = CompactionManager.instance.submitBackground(this); - if (waitForFutures) - FBUtilities.waitOnFutures(futures); + strategyContainer.enable(); + Future future = CompactionManager.instance.submitBackground(this); + if (waitForFuture) + FBUtilities.waitOnFuture(future); } public boolean isAutoCompactionDisabled() { - return !this.compactionStrategyManager.isEnabled(); + return !this.strategyContainer.isEnabled(); } - /* - JMX getters and setters for the Defaults. - - get/set minCompactionThreshold - - get/set maxCompactionThreshold - - get memsize - - get memops - - get/set memtime + public SortedLocalRanges getLocalRanges() + { + synchronized (this) + { + if (localRanges != null && !localRanges.isOutOfDate()) + return localRanges; + + localRanges = SortedLocalRanges.create(this); + return localRanges; + } + } + + /** + * Return the compaction strategy for this CFS. Even though internally the strategy container + * implements the strategy, we would like to just expose {@link CompactionStrategy} externally. + * This is not currently possible for the reasons explained in {@link this#getCompactionStrategyContainer()}, + * so we expose the container as well, but using a separate method, marked as deprecated. + * + * @return the compaction strategy for this CFS + */ + public CompactionStrategy getCompactionStrategy() + { + return strategyContainer; + } + + /** + * The reasons for exposing the compaction strategy container are the following: + * + * - Unit tests + * - Repair + * + * Eventually we would like to only expose the {@link CompactionStrategy}, so for new code call + * {@link this#getCompactionStrategy()} instead. + * + * @return the compaction strategy container */ + @Deprecated + @VisibleForTesting + public CompactionStrategyContainer getCompactionStrategyContainer() + { + return strategyContainer; + } - public CompactionStrategyManager getCompactionStrategyManager() + /** + * This option determines if tombstones should only be removed when the sstable has been repaired. + * Because this option was introduced in patch releases (I'm guessing), the compaction parameters were + * abused. Eventually this option should be moved out of the compaction parameters. TODO: move it + * to the new compaction strategy interface. + * + * @return true if tombstones can only be removed if the sstable has been repaired + */ + public boolean onlyPurgeRepairedTombstones() { - return compactionStrategyManager; + // Here we need to ask the CSM for the parameters in case they were changed over JMX without changing the schema, + // for now the CSM has the up-to-date copy of the params + CompactionParams params = strategyContainer.getCompactionParams(); + return Boolean.parseBoolean(params.options().get(CompactionStrategyOptions.ONLY_PURGE_REPAIRED_TOMBSTONES)); } public void setCrcCheckChance(double crcCheckChance) @@ -2579,6 +2770,21 @@ public double getMeanPartitionSize() return count > 0 ? sum * 1.0 / count : 0; } + public double sstablePartitionReadLatency() + { + return metric == null ? 0 : metric.sstablePartitionReadLatency.get(); + } + + public double getCompactionTimePerKb() + { + return metric == null ? 0 : metric.compactionTimePerKb.get(); + } + + public double getFlushTimePerKb() + { + return metric == null ? 0 : metric.flushTimePerKb.get(); + } + public int getMeanRowCount() { long totalRows = 0; @@ -2610,6 +2816,77 @@ public DecoratedKey decorateKey(ByteBuffer key) return getPartitioner().decorateKey(key); } + public BloomFilterTracker getBloomFilterTracker() + { + return bloomFilterTracker; + } + + public long getBloomFilterFalsePositiveCount() + { + return bloomFilterTracker.getFalsePositiveCount(); + } + + public long getBloomFilterTruePositiveCount() + { + return bloomFilterTracker.getTruePositiveCount(); + } + + public long getBloomFilterTrueNegativeCount() + { + return bloomFilterTracker.getTrueNegativeCount(); + } + + public double getRecentBloomFilterFalsePositiveRate() + { + return bloomFilterTracker.getRecentFalsePositiveRate(); + } + + public double getRecentBloomFilterTruePositiveRate() + { + return bloomFilterTracker.getRecentTruePositiveRate(); + } + + public double getRecentBloomFilterTrueNegativeRate() + { + return bloomFilterTracker.getRecentTrueNegativeRate(); + } + + public double bloomFilterFpRatio() + { + return metric == null ? 0 : metric.bloomFilterFalseRatio.getValue(); + } + + public long getReadRequests() + { + return metric == null ? 0 : metric.readRequests.getCount(); + } + + public long getBytesInserted() + { + return metric == null ? 0 : metric.bytesInserted.getCount(); + } + + /** + * @return the write amplification (bytes flushed + bytes compacted / bytes flushed). + */ + public double getWA() + { + if (metric == null) + return 0; + + double bytesCompacted = metric.compactionBytesWritten.getCount(); + double bytesFlushed = metric.bytesFlushed.getCount(); + return bytesFlushed <= 0 ? 0 : (bytesFlushed + bytesCompacted) / bytesFlushed; + } + + public double getFlushSizeOnDisk() + { + if (metric == null) + return 0; + + return metric.flushSizeOnDisk.get(); + } + /** true if this CFS contains secondary index data */ public boolean isIndex() { @@ -2630,17 +2907,20 @@ public List getBuiltIndexes() public int getUnleveledSSTables() { - return compactionStrategyManager.getUnleveledSSTables(); + if (strategyContainer instanceof CompactionStrategyManager) + return ((CompactionStrategyManager) strategyContainer).getUnleveledSSTables(); + else + return 0; } public int[] getSSTableCountPerLevel() { - return compactionStrategyManager.getSSTableCountPerLevel(); + return strategyContainer.getSSTableCountPerLevel(); } public int getLevelFanoutSize() { - return compactionStrategyManager.getLevelFanoutSize(); + return strategyContainer.getLevelFanoutSize(); } public static class ViewFragment @@ -2801,8 +3081,14 @@ public DiskBoundaries getDiskBoundaries() return diskBoundaryManager.getDiskBoundaries(this); } - public void invalidateDiskBoundaries() + public void invalidateLocalRangesAndDiskBoundaries() { + synchronized (this) + { + if (localRanges != null) + localRanges.invalidate(); + } + diskBoundaryManager.invalidate(); } @@ -2927,4 +3213,84 @@ public boolean hasMisplacedSSTables() } return false; } + + private static void verifyMetadata(SSTableReader sstable, long repairedAt, UUID pendingRepair, boolean isTransient) + { + if (!Objects.equals(pendingRepair, sstable.getPendingRepair())) + throw new IllegalStateException(String.format("Failed setting pending repair to %s on %s (pending repair is %s)", pendingRepair, sstable, sstable.getPendingRepair())); + if (repairedAt != sstable.getRepairedAt()) + throw new IllegalStateException(String.format("Failed setting repairedAt to %d on %s (repairedAt is %d)", repairedAt, sstable, sstable.getRepairedAt())); + if (isTransient != sstable.isTransient()) + throw new IllegalStateException(String.format("Failed setting isTransient to %b on %s (isTransient is %b)", isTransient, sstable, sstable.isTransient())); + } + + /** + * This method is exposed for testing only + * NotThreadSafe + */ + @VisibleForTesting + public int mutateRepaired(Collection sstables, long repairedAt, UUID pendingRepair, boolean isTransient) throws IOException + { + Set changed = new HashSet<>(); + try + { + for (SSTableReader sstable: sstables) + { + sstable.mutateRepairedAndReload(repairedAt, pendingRepair, isTransient); + verifyMetadata(sstable, repairedAt, pendingRepair, isTransient); + changed.add(sstable); + } + } + finally + { + // if there was an exception mutating repairedAt, we should still notify for the + // sstables that we were able to modify successfully before releasing the lock + getTracker().notifySSTableRepairedStatusChanged(changed); + } + return changed.size(); + } + + /** + * Mutates sstable repairedAt times and notifies listeners of the change with the writeLock held. Prevents races + * with other processes between when the metadata is changed and when sstables are moved between strategies. + */ + public int mutateRepaired(@Nullable final ReentrantReadWriteLock.WriteLock writeLock, + Collection sstables, + long repairedAt, + UUID pendingRepair, + boolean isTransient) throws IOException + { + if (writeLock == null) + return mutateRepaired(sstables, repairedAt, pendingRepair, isTransient); + + writeLock.lock(); + try + { + return mutateRepaired(sstables, repairedAt, pendingRepair, isTransient); + } + finally + { + writeLock.unlock(); + } + } + + public boolean hasPendingRepairSSTables(UUID sessionID) + { + return Iterables.any(data.getLiveSSTables(), pendingRepairPredicate(sessionID)); + } + + public Set getPendingRepairSSTables(UUID sessionID) + { + return Sets.filter(data.getLiveSSTables(), pendingRepairPredicate(sessionID)); + } + + public static Predicate pendingRepairPredicate(@Nonnull UUID sessionID) + { + return sstable -> sstable.getPendingRepair() != null && sessionID.equals(sstable.getPendingRepair()); + } + + public static Predicate nonSuspectAndNotInPredicate(Set compacting) + { + return sstable -> !sstable.isMarkedSuspect() && !compacting.contains(sstable); + } } diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java index 0360e344d16a..634bf1e3141e 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java @@ -191,19 +191,20 @@ public List importNewSSTables(Set srcPaths, @Deprecated public void loadNewSSTables(); + /** * @return the number of SSTables in L0. Always return 0 if Leveled compaction is not enabled. */ public int getUnleveledSSTables(); /** - * @return sstable count for each level. null unless leveled compaction is used. + * @return sstable count for each level. empty unless leveled or unified compaction is used. * array index corresponds to level(int[0] is for level 0, ...). */ public int[] getSSTableCountPerLevel(); /** - * @return sstable fanout size for level compaction strategy. + * @return sstable fanout size for level or unified compaction strategies. Default LCS fanout size otherwise. */ public int getLevelFanoutSize(); diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java index fbaf3fd4d65a..8cbfb47a425c 100644 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@ -17,14 +17,19 @@ */ package org.apache.cassandra.db; +import java.util.Locale; + +import javax.annotation.Nullable; import com.carrotsearch.hppc.ObjectIntHashMap; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.locator.Endpoints; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.NetworkTopologyStrategy; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.ProtocolException; import static org.apache.cassandra.locator.Replicas.addToCountPerDc; @@ -74,6 +79,11 @@ private ConsistencyLevel(int code, boolean isDCLocal) this.isDCLocal = isDCLocal; } + public static ConsistencyLevel fromString(String str) + { + return valueOf(str.toUpperCase(Locale.US)); + } + public static ConsistencyLevel fromCode(int code) { if (code < 0 || code >= codeIdx.length) @@ -207,8 +217,10 @@ public void validateForRead() throws InvalidRequestException } } - public void validateForWrite() throws InvalidRequestException + public void validateForWrite(String keyspaceName, QueryState queryState) throws InvalidRequestException { + Guardrails.disallowedWriteConsistencies.ensureAllowed(this, queryState); + switch (this) { case SERIAL: @@ -218,8 +230,10 @@ public void validateForWrite() throws InvalidRequestException } // This is the same than validateForWrite really, but we include a slightly different error message for SERIAL/LOCAL_SERIAL - public void validateForCasCommit(AbstractReplicationStrategy replicationStrategy) throws InvalidRequestException + public void validateForCasCommit(AbstractReplicationStrategy replicationStrategy, String keyspaceName, QueryState queryState) throws InvalidRequestException { + Guardrails.disallowedWriteConsistencies.ensureAllowed(this, queryState); + switch (this) { case EACH_QUORUM: @@ -231,8 +245,10 @@ public void validateForCasCommit(AbstractReplicationStrategy replicationStrategy } } - public void validateForCas() throws InvalidRequestException + public void validateForCas(String keyspaceName, QueryState queryState) throws InvalidRequestException { + Guardrails.disallowedWriteConsistencies.ensureAllowed(this, queryState); + if (!isSerialConsistency()) throw new InvalidRequestException("Invalid consistency for conditional update. Must be one of SERIAL or LOCAL_SERIAL"); } @@ -242,8 +258,10 @@ public boolean isSerialConsistency() return this == SERIAL || this == LOCAL_SERIAL; } - public void validateCounterForWrite(TableMetadata metadata) throws InvalidRequestException + public void validateCounterForWrite(TableMetadata metadata, QueryState queryState) throws InvalidRequestException { + Guardrails.disallowedWriteConsistencies.ensureAllowed(this, queryState); + if (this == ConsistencyLevel.ANY) throw new InvalidRequestException("Consistency level ANY is not yet supported for counter table " + metadata.name); @@ -257,4 +275,21 @@ private void requireNetworkTopologyStrategy(AbstractReplicationStrategy replicat throw new InvalidRequestException(String.format("consistency level %s not compatible with replication strategy (%s)", this, replicationStrategy.getClass().getName())); } + + /** + * Returns the strictest consistency level allowed by Guardrails. + * + * @param state the query state, used to skip the guardrails check if the query is internal or is done by a superuser. + * @return the strictest allowed serial consistency level + * @throws InvalidRequestException if all serial consistency level are disallowed + */ + public static ConsistencyLevel defaultSerialConsistency(@Nullable QueryState state) throws InvalidRequestException + { + if (DatabaseDescriptor.getRawConfig() == null || !Guardrails.disallowedWriteConsistencies.triggersOn(ConsistencyLevel.SERIAL, state)) + return ConsistencyLevel.SERIAL; + else if (!Guardrails.disallowedWriteConsistencies.triggersOn(ConsistencyLevel.LOCAL_SERIAL, state)) + return ConsistencyLevel.LOCAL_SERIAL; + + throw new InvalidRequestException("Serial consistency levels are disallowed by disallowedWriteConsistencies Guardrail"); + } } diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java index fe1e46e0b1c6..70058886d8af 100644 --- a/src/java/org/apache/cassandra/db/CounterMutation.java +++ b/src/java/org/apache/cassandra/db/CounterMutation.java @@ -47,6 +47,7 @@ import org.apache.cassandra.utils.btree.BTreeSet; import static java.util.concurrent.TimeUnit.*; +import static org.apache.cassandra.net.MessagingService.VERSION_SG_10; import static org.apache.cassandra.net.MessagingService.VERSION_30; import static org.apache.cassandra.net.MessagingService.VERSION_3014; import static org.apache.cassandra.net.MessagingService.VERSION_40; @@ -326,6 +327,7 @@ public long getTimeout(TimeUnit unit) private int serializedSize30; private int serializedSize3014; private int serializedSize40; + private int serializedSizeSG10; public int serializedSize(int version) { @@ -343,6 +345,10 @@ public int serializedSize(int version) if (serializedSize40 == 0) serializedSize40 = (int) serializer.serializedSize(this, VERSION_40); return serializedSize40; + case VERSION_SG_10: + if (serializedSizeSG10 == 0) + serializedSizeSG10 = (int) serializer.serializedSize(this, VERSION_SG_10); + return serializedSizeSG10; default: throw new IllegalStateException("Unknown serialization version: " + version); } diff --git a/src/java/org/apache/cassandra/db/DecoratedKey.java b/src/java/org/apache/cassandra/db/DecoratedKey.java index 92d641460e10..ce022bc074ed 100644 --- a/src/java/org/apache/cassandra/db/DecoratedKey.java +++ b/src/java/org/apache/cassandra/db/DecoratedKey.java @@ -19,13 +19,17 @@ import java.nio.ByteBuffer; import java.util.Comparator; +import java.util.function.BiFunction; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; import org.apache.cassandra.dht.Token.KeyBound; import org.apache.cassandra.utils.ByteBufferUtil; -import org.apache.cassandra.utils.MurmurHash; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.apache.cassandra.utils.IFilter.FilterKey; +import org.apache.cassandra.utils.MurmurHash; /** * Represents a decorated key, handy for certain operations @@ -97,6 +101,22 @@ public static int compareTo(IPartitioner partitioner, ByteBuffer key, PartitionP return cmp == 0 ? ByteBufferUtil.compareUnsigned(key, otherKey.getKey()) : cmp; } + @Override + public ByteSource asComparableBytes(Version version) + { + // Note: In the legacy version one encoding could be a prefix of another as the escaping is only weakly + // prefix-free (see ByteSourceTest.testDecoratedKeyPrefixes()). + // The OSS41 version avoids this by adding a terminator. + return ByteSource.withTerminator(version == Version.LEGACY ? ByteSource.END_OF_STREAM : ByteSource.TERMINATOR, + token.asComparableBytes(version), + keyComparableBytes(version)); + } + + protected ByteSource keyComparableBytes(Version version) + { + return ByteSource.of(getKey(), version); + } + public IPartitioner getPartitioner() { return getToken().getPartitioner(); @@ -131,10 +151,40 @@ public Token getToken() } public abstract ByteBuffer getKey(); + public abstract int getKeyLength(); public void filterHash(long[] dest) { ByteBuffer key = getKey(); MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0, dest); } + + /** + * A template factory method for creating decorated keys from their byte-comparable representation. + */ + static T fromByteComparable(ByteComparable byteComparable, + Version version, + IPartitioner partitioner, + BiFunction decoratedKeyFactory) + { + ByteSource.Peekable peekable = byteComparable.asPeekableBytes(version); + // Decode the token from the first component of the multi-component sequence representing the whole decorated key. + Token token = partitioner.getTokenFactory().fromComparableBytes(ByteSourceInverse.nextComponentSource(peekable), version); + // Decode the key bytes from the second component. + byte[] keyBytes = ByteSourceInverse.getUnescapedBytes(ByteSourceInverse.nextComponentSource(peekable)); + // Instantiate a decorated key from the decoded token and key bytes, using the provided factory method. + return decoratedKeyFactory.apply(token, keyBytes); + } + + public static byte[] keyFromByteComparable(ByteComparable byteComparable, + Version version, + IPartitioner partitioner) + { + ByteSource.Peekable peekable = byteComparable.asPeekableBytes(version); + // Decode the token from the first component of the multi-component sequence representing the whole decorated key. + // We won't use it, but the decoding also positions the byte source after it. + partitioner.getTokenFactory().fromComparableBytes(ByteSourceInverse.nextComponentSource(peekable), version); + // Decode the key bytes from the second component. + return ByteSourceInverse.getUnescapedBytes(ByteSourceInverse.nextComponentSource(peekable)); + } } diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java index d8ac91db9834..718fcf7a144b 100644 --- a/src/java/org/apache/cassandra/db/DeletionTime.java +++ b/src/java/org/apache/cassandra/db/DeletionTime.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db; import java.io.IOException; +import java.nio.ByteBuffer; import com.google.common.base.Objects; @@ -180,6 +181,15 @@ public DeletionTime deserialize(DataInputPlus in) throws IOException : new DeletionTime(mfda, ldt); } + public DeletionTime deserialize(ByteBuffer buf, int offset) + { + int ldt = buf.getInt(offset); + long mfda = buf.getLong(offset + 4); + return mfda == Long.MIN_VALUE && ldt == Integer.MAX_VALUE + ? LIVE + : new DeletionTime(mfda, ldt); + } + public void skip(DataInputPlus in) throws IOException { in.skipBytesFully(4 + 8); diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java index cf4238c67de5..83c7cf92db66 100644 --- a/src/java/org/apache/cassandra/db/Directories.java +++ b/src/java/org/apache/cassandra/db/Directories.java @@ -621,6 +621,16 @@ public long getAvailableSpace() return availableSpace > 0 ? availableSpace : 0; } + public long getTotalSpace() + { + return FileUtils.getTotalSpace(location); + } + + public long getSpaceUsed() + { + return getTotalSpace() - getAvailableSpace(); + } + @Override public boolean equals(Object o) { diff --git a/src/java/org/apache/cassandra/db/DiskBoundaries.java b/src/java/org/apache/cassandra/db/DiskBoundaries.java index f33b43eb3d80..30d2aada44e6 100644 --- a/src/java/org/apache/cassandra/db/DiskBoundaries.java +++ b/src/java/org/apache/cassandra/db/DiskBoundaries.java @@ -20,34 +20,41 @@ import java.util.Collections; import java.util.List; +import java.util.Objects; + +import javax.annotation.Nullable; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.service.StorageService; public class DiskBoundaries { - public final List directories; - public final ImmutableList positions; - final long ringVersion; + @Nullable public final List directories; + @Nullable private final ImmutableList positions; + public final SortedLocalRanges localRanges; final int directoriesVersion; private final ColumnFamilyStore cfs; private volatile boolean isInvalid = false; - public DiskBoundaries(ColumnFamilyStore cfs, Directories.DataDirectory[] directories, int diskVersion) + public DiskBoundaries(ColumnFamilyStore cfs, + @Nullable Directories.DataDirectory[] directories, + SortedLocalRanges localRanges, + int diskVersion) { - this(cfs, directories, null, -1, diskVersion); + this(cfs, directories, null, localRanges, diskVersion); } - @VisibleForTesting - public DiskBoundaries(ColumnFamilyStore cfs, Directories.DataDirectory[] directories, List positions, long ringVersion, int diskVersion) + public DiskBoundaries(ColumnFamilyStore cfs, + @Nullable Directories.DataDirectory[] directories, + @Nullable List positions, + SortedLocalRanges localRanges, + int diskVersion) { this.directories = directories == null ? null : ImmutableList.copyOf(directories); this.positions = positions == null ? null : ImmutableList.copyOf(positions); - this.ringVersion = ringVersion; + this.localRanges = localRanges; this.directoriesVersion = diskVersion; this.cfs = cfs; } @@ -59,17 +66,17 @@ public boolean equals(Object o) DiskBoundaries that = (DiskBoundaries) o; - if (ringVersion != that.ringVersion) return false; - if (directoriesVersion != that.directoriesVersion) return false; - if (!directories.equals(that.directories)) return false; - return positions != null ? positions.equals(that.positions) : that.positions == null; + return Objects.equals(localRanges, that.localRanges) && + directoriesVersion == that.directoriesVersion && + Objects.equals(directories, that.directories) && + Objects.equals(positions, that.positions); } public int hashCode() { int result = directories != null ? directories.hashCode() : 0; result = 31 * result + (positions != null ? positions.hashCode() : 0); - result = 31 * result + (int) (ringVersion ^ (ringVersion >>> 32)); + result = 31 * result + localRanges.hashCode(); result = 31 * result + directoriesVersion; return result; } @@ -79,7 +86,7 @@ public String toString() return "DiskBoundaries{" + "directories=" + directories + ", positions=" + positions + - ", ringVersion=" + ringVersion + + ", localRanges=" + localRanges.toString() + ", directoriesVersion=" + directoriesVersion + '}'; } @@ -91,9 +98,9 @@ public boolean isOutOfDate() { if (isInvalid) return true; + int currentDiskVersion = DisallowedDirectories.getDirectoriesVersion(); - long currentRingVersion = StorageService.instance.getTokenMetadata().getRingVersion(); - return currentDiskVersion != directoriesVersion || (ringVersion != -1 && currentRingVersion != ringVersion); + return currentDiskVersion != directoriesVersion || localRanges.isOutOfDate(); } public void invalidate() @@ -101,7 +108,7 @@ public void invalidate() this.isInvalid = true; } - public int getDiskIndex(SSTableReader sstable) + public int getDiskIndexFromKey(SSTableReader sstable) { if (positions == null) { @@ -130,7 +137,7 @@ public int getBoundariesFromSSTableDirectory(Descriptor descriptor) public Directories.DataDirectory getCorrectDiskForSSTable(SSTableReader sstable) { - return directories.get(getDiskIndex(sstable)); + return directories.get(getDiskIndexFromKey(sstable)); } public Directories.DataDirectory getCorrectDiskForKey(DecoratedKey key) @@ -138,20 +145,55 @@ public Directories.DataDirectory getCorrectDiskForKey(DecoratedKey key) if (positions == null) return null; - return directories.get(getDiskIndex(key)); + return directories.get(getDiskIndexFromKey(key)); } public boolean isInCorrectLocation(SSTableReader sstable, Directories.DataDirectory currentLocation) { - int diskIndex = getDiskIndex(sstable); + int diskIndex = getDiskIndexFromKey(sstable); PartitionPosition diskLast = positions.get(diskIndex); return directories.get(diskIndex).equals(currentLocation) && sstable.last.compareTo(diskLast) <= 0; } - private int getDiskIndex(DecoratedKey key) + /** + * Return the number of boundaries. If this instance was created with token boundaries (positions) then this + * is the number of boundaries. If this instance was created without boundaries but only with directories, then + * this is the number of directories. + * + * @return the number of boundaries. + */ + public int getNumBoundaries() + { + return positions == null ? directories.size() : positions.size(); + } + + private int getDiskIndexFromKey(DecoratedKey key) { int pos = Collections.binarySearch(positions, key); assert pos < 0; return -pos - 1; } + + /** + * Return the local sorted ranges, which contain the local ranges for this node, sorted. + * See {@link SortedLocalRanges}. + * + * @return the local ranges, see {@link SortedLocalRanges}. + */ + public SortedLocalRanges getLocalRanges() + { + return localRanges; + } + + /** + * Returns a non-modifiable list of the disk boundary positions. This will be null if the token space is not split + * for the disks, this is not normally the case). + * + * Extracted as a method (instead of direct access to the final field) to permit mocking in tests. + */ + @Nullable + public List getPositions() + { + return positions; + } } diff --git a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java index cc617da702e2..04c6384b2a2d 100644 --- a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java +++ b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java @@ -19,23 +19,15 @@ package org.apache.cassandra.db; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; -import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.IPartitioner; -import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Splitter; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.locator.RangesAtEndpoint; -import org.apache.cassandra.locator.TokenMetadata; -import org.apache.cassandra.service.PendingRangeCalculatorService; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.FBUtilities; public class DiskBoundaryManager { @@ -44,8 +36,6 @@ public class DiskBoundaryManager public DiskBoundaries getDiskBoundaries(ColumnFamilyStore cfs) { - if (!cfs.getPartitioner().splitter().isPresent()) - return new DiskBoundaries(cfs, cfs.getDirectories().getWriteableLocations(), DisallowedDirectories.getDirectoriesVersion()); if (diskBoundaries == null || diskBoundaries.isOutOfDate()) { synchronized (this) @@ -53,8 +43,13 @@ public DiskBoundaries getDiskBoundaries(ColumnFamilyStore cfs) if (diskBoundaries == null || diskBoundaries.isOutOfDate()) { logger.debug("Refreshing disk boundary cache for {}.{}", cfs.keyspace.getName(), cfs.getTableName()); + SortedLocalRanges localRanges = cfs.getLocalRanges(); + DiskBoundaries oldBoundaries = diskBoundaries; - diskBoundaries = getDiskBoundaryValue(cfs); + diskBoundaries = !cfs.getPartitioner().splitter().isPresent() + ? new DiskBoundaries(cfs, cfs.getDirectories().getWriteableLocations(), localRanges, DisallowedDirectories.getDirectoriesVersion()) + : getDiskBoundaryValue(cfs, localRanges); + logger.debug("Updating boundaries from {} to {} for {}.{}", oldBoundaries, diskBoundaries, cfs.keyspace.getName(), cfs.getTableName()); } } @@ -68,34 +63,9 @@ public void invalidate() diskBoundaries.invalidate(); } - private static DiskBoundaries getDiskBoundaryValue(ColumnFamilyStore cfs) - { - RangesAtEndpoint localRanges; - - long ringVersion; - TokenMetadata tmd; - do - { - tmd = StorageService.instance.getTokenMetadata(); - ringVersion = tmd.getRingVersion(); - if (StorageService.instance.isBootstrapMode() - && !StorageService.isReplacingSameAddress()) // When replacing same address, the node marks itself as UN locally - { - PendingRangeCalculatorService.instance.blockUntilFinished(); - localRanges = tmd.getPendingRanges(cfs.keyspace.getName(), FBUtilities.getBroadcastAddressAndPort()); - } - else - { - // Reason we use use the future settled TMD is that if we decommission a node, we want to stream - // from that node to the correct location on disk, if we didn't, we would put new files in the wrong places. - // We do this to minimize the amount of data we need to move in rebalancedisks once everything settled - localRanges = cfs.keyspace.getReplicationStrategy().getAddressReplicas(tmd.cloneAfterAllSettled(), FBUtilities.getBroadcastAddressAndPort()); - } - logger.debug("Got local ranges {} (ringVersion = {})", localRanges, ringVersion); - } - while (ringVersion != tmd.getRingVersion()); // if ringVersion is different here it means that - // it might have changed before we calculated localRanges - recalculate + private static DiskBoundaries getDiskBoundaryValue(ColumnFamilyStore cfs, SortedLocalRanges localRanges) + { int directoriesVersion; Directories.DataDirectory[] dirs; do @@ -105,12 +75,11 @@ private static DiskBoundaries getDiskBoundaryValue(ColumnFamilyStore cfs) } while (directoriesVersion != DisallowedDirectories.getDirectoriesVersion()); // if directoriesVersion has changed we need to recalculate - if (localRanges == null || localRanges.isEmpty()) - return new DiskBoundaries(cfs, dirs, null, ringVersion, directoriesVersion); + if (localRanges == null || localRanges.getRanges().isEmpty()) + return new DiskBoundaries(cfs, dirs, null, localRanges, directoriesVersion); - List positions = getDiskBoundaries(localRanges, cfs.getPartitioner(), dirs); - - return new DiskBoundaries(cfs, dirs, positions, ringVersion, directoriesVersion); + List positions = getDiskBoundaries(localRanges.getRanges(), cfs.getPartitioner(), dirs); + return new DiskBoundaries(cfs, dirs, positions, localRanges, directoriesVersion); } /** @@ -122,27 +91,15 @@ private static DiskBoundaries getDiskBoundaryValue(ColumnFamilyStore cfs) * * The final entry in the returned list will always be the partitioner maximum tokens upper key bound */ - private static List getDiskBoundaries(RangesAtEndpoint replicas, IPartitioner partitioner, Directories.DataDirectory[] dataDirectories) + private static List getDiskBoundaries(List weightedRanges, IPartitioner partitioner, Directories.DataDirectory[] dataDirectories) { assert partitioner.splitter().isPresent(); Splitter splitter = partitioner.splitter().get(); - boolean dontSplitRanges = DatabaseDescriptor.getNumTokens() > 1; - - List weightedRanges = new ArrayList<>(replicas.size()); - // note that Range.sort unwraps any wraparound ranges, so we need to sort them here - for (Range r : Range.sort(replicas.onlyFull().ranges())) - weightedRanges.add(new Splitter.WeightedRange(1.0, r)); - - for (Range r : Range.sort(replicas.onlyTransient().ranges())) - weightedRanges.add(new Splitter.WeightedRange(0.1, r)); - - weightedRanges.sort(Comparator.comparing(Splitter.WeightedRange::left)); + Splitter.SplitType splitType = DatabaseDescriptor.getNumTokens() > 1 ? Splitter.SplitType.PREFER_WHOLE : Splitter.SplitType.ALWAYS_SPLIT; - List boundaries = splitter.splitOwnedRanges(dataDirectories.length, weightedRanges, dontSplitRanges); - // If we can't split by ranges, split evenly to ensure utilisation of all disks - if (dontSplitRanges && boundaries.size() < dataDirectories.length) - boundaries = splitter.splitOwnedRanges(dataDirectories.length, weightedRanges, false); + List boundaries = splitter.splitOwnedRanges(dataDirectories.length, weightedRanges, splitType).boundaries; + assert boundaries.size() == dataDirectories.length : "Wrong number of boundaries for directories: " + boundaries.size(); List diskBoundaries = new ArrayList<>(); for (int i = 0; i < boundaries.size() - 1; i++) diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index ead01fbe241e..46e46688496b 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -51,7 +51,6 @@ import org.apache.cassandra.exceptions.WriteTimeoutException; import org.apache.cassandra.index.Index; import org.apache.cassandra.index.SecondaryIndexManager; -import org.apache.cassandra.index.transactions.UpdateTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.metrics.KeyspaceMetrics; @@ -66,7 +65,6 @@ import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.ByteBufferUtil; -import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.concurrent.OpOrder; @@ -388,12 +386,12 @@ private void createReplicationStrategy(KeyspaceMetadata ksm) if (!ksm.params.replication.equals(replicationParams)) { logger.debug("New replication settings for keyspace {} - invalidating disk boundary caches", ksm.name); - columnFamilyStores.values().forEach(ColumnFamilyStore::invalidateDiskBoundaries); + columnFamilyStores.values().forEach(ColumnFamilyStore::invalidateLocalRangesAndDiskBoundaries); } replicationParams = ksm.params.replication; } - // best invoked on the compaction mananger. + // best invoked on the compaction manager. public void dropCf(TableId tableId) { assert columnFamilyStores.containsKey(tableId); @@ -401,7 +399,6 @@ public void dropCf(TableId tableId) if (cfs == null) return; - cfs.getCompactionStrategyManager().shutdown(); CompactionManager.instance.interruptCompactionForCFs(cfs.concatWithIndexes(), (sstable) -> true, true); // wait for any outstanding reads/writes that might affect the CFS cfs.keyspace.writeOrder.awaitNewBarrier(); @@ -413,7 +410,7 @@ public void dropCf(TableId tableId) // disassociate a cfs from this keyspace instance. private void unloadCf(ColumnFamilyStore cfs) { - cfs.forceBlockingFlush(); + cfs.unloadCf(); cfs.invalidate(); } @@ -626,7 +623,6 @@ else if (isDeferrable) columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, MILLISECONDS); } } - int nowInSec = FBUtilities.nowInSeconds(); try (WriteContext ctx = getWriteHandler().beginWrite(mutation, makeDurable)) { for (PartitionUpdate upd : mutation.getPartitionUpdates()) @@ -655,10 +651,7 @@ else if (isDeferrable) } } - UpdateTransaction indexTransaction = updateIndexes - ? cfs.indexManager.newUpdateTransaction(upd, ctx, nowInSec) - : UpdateTransaction.NO_OP; - cfs.getWriteHandler().write(upd, ctx, indexTransaction); + cfs.getWriteHandler().write(upd, ctx, updateIndexes); if (requiresViewUpdate) baseComplete.set(System.currentTimeMillis()); @@ -685,11 +678,11 @@ public AbstractReplicationStrategy getReplicationStrategy() return replicationStrategy; } - public List> flush() + public List> flush(ColumnFamilyStore.FlushReason reason) { List> futures = new ArrayList<>(columnFamilyStores.size()); for (ColumnFamilyStore cfs : columnFamilyStores.values()) - futures.add(cfs.forceFlush()); + futures.add(cfs.forceFlush(reason)); return futures; } diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java deleted file mode 100644 index 73c64169f712..000000000000 --- a/src/java/org/apache/cassandra/db/Memtable.java +++ /dev/null @@ -1,664 +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.db; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.apache.cassandra.db.commitlog.CommitLogPosition; -import org.apache.cassandra.db.commitlog.IntervalSet; -import org.apache.cassandra.db.filter.ClusteringIndexFilter; -import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; -import org.apache.cassandra.db.partitions.AbstractBTreePartition; -import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; -import org.apache.cassandra.db.partitions.AtomicBTreePartition; -import org.apache.cassandra.db.partitions.Partition; -import org.apache.cassandra.db.partitions.PartitionUpdate; -import org.apache.cassandra.db.rows.EncodingStats; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; -import org.apache.cassandra.dht.AbstractBounds; -import org.apache.cassandra.dht.Bounds; -import org.apache.cassandra.dht.IncludingExcludingBounds; -import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.index.transactions.UpdateTransaction; -import org.apache.cassandra.io.sstable.Descriptor; -import org.apache.cassandra.io.sstable.SSTableMultiWriter; -import org.apache.cassandra.io.sstable.metadata.MetadataCollector; -import org.apache.cassandra.schema.ColumnMetadata; -import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.ActiveRepairService; -import org.apache.cassandra.utils.ByteBufferUtil; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.ObjectSizes; -import org.apache.cassandra.utils.concurrent.OpOrder; -import org.apache.cassandra.utils.memory.HeapPool; -import org.apache.cassandra.utils.memory.MemtableAllocator; -import org.apache.cassandra.utils.memory.MemtableCleaner; -import org.apache.cassandra.utils.memory.MemtablePool; -import org.apache.cassandra.utils.memory.NativePool; -import org.apache.cassandra.utils.memory.SlabPool; - -public class Memtable implements Comparable -{ - private static final Logger logger = LoggerFactory.getLogger(Memtable.class); - - public static final MemtablePool MEMORY_POOL = createMemtableAllocatorPool(); - - private static MemtablePool createMemtableAllocatorPool() - { - long heapLimit = DatabaseDescriptor.getMemtableHeapSpaceInMb() << 20; - long offHeapLimit = DatabaseDescriptor.getMemtableOffheapSpaceInMb() << 20; - final float cleaningThreshold = DatabaseDescriptor.getMemtableCleanupThreshold(); - final MemtableCleaner cleaner = ColumnFamilyStore::flushLargestMemtable; - switch (DatabaseDescriptor.getMemtableAllocationType()) - { - case unslabbed_heap_buffers: - return new HeapPool(heapLimit, cleaningThreshold, cleaner); - case heap_buffers: - return new SlabPool(heapLimit, 0, cleaningThreshold, cleaner); - case offheap_buffers: - return new SlabPool(heapLimit, offHeapLimit, cleaningThreshold, cleaner); - case offheap_objects: - return new NativePool(heapLimit, offHeapLimit, cleaningThreshold, cleaner); - default: - throw new AssertionError(); - } - } - - private static final int ROW_OVERHEAD_HEAP_SIZE = estimateRowOverhead(Integer.parseInt(System.getProperty("cassandra.memtable_row_overhead_computation_step", "100000"))); - - private final MemtableAllocator allocator; - private final AtomicLong liveDataSize = new AtomicLong(0); - private final AtomicLong currentOperations = new AtomicLong(0); - - // the write barrier for directing writes to this memtable or the next during a switch - private volatile OpOrder.Barrier writeBarrier; - // the precise upper bound of CommitLogPosition owned by this memtable - private volatile AtomicReference commitLogUpperBound; - // the precise lower bound of CommitLogPosition owned by this memtable; equal to its predecessor's commitLogUpperBound - private AtomicReference commitLogLowerBound; - - // The approximate lower bound by this memtable; must be <= commitLogLowerBound once our predecessor - // has been finalised, and this is enforced in the ColumnFamilyStore.setCommitLogUpperBound - private final CommitLogPosition approximateCommitLogLowerBound = CommitLog.instance.getCurrentPosition(); - - public int compareTo(Memtable that) - { - return this.approximateCommitLogLowerBound.compareTo(that.approximateCommitLogLowerBound); - } - - public static final class LastCommitLogPosition extends CommitLogPosition - { - public LastCommitLogPosition(CommitLogPosition copy) - { - super(copy.segmentId, copy.position); - } - } - - // We index the memtable by PartitionPosition only for the purpose of being able - // to select key range using Token.KeyBound. However put() ensures that we - // actually only store DecoratedKey. - private final ConcurrentNavigableMap partitions = new ConcurrentSkipListMap<>(); - public final ColumnFamilyStore cfs; - private final long creationNano = System.nanoTime(); - - // The smallest timestamp for all partitions stored in this memtable - private long minTimestamp = Long.MAX_VALUE; - - // Record the comparator of the CFS at the creation of the memtable. This - // is only used when a user update the CF comparator, to know if the - // memtable was created with the new or old comparator. - public final ClusteringComparator initialComparator; - - private final ColumnsCollector columnsCollector; - private final StatsCollector statsCollector = new StatsCollector(); - - // only to be used by init(), to setup the very first memtable for the cfs - public Memtable(AtomicReference commitLogLowerBound, ColumnFamilyStore cfs) - { - this.cfs = cfs; - this.commitLogLowerBound = commitLogLowerBound; - this.allocator = MEMORY_POOL.newAllocator(); - this.initialComparator = cfs.metadata().comparator; - this.cfs.scheduleFlush(); - this.columnsCollector = new ColumnsCollector(cfs.metadata().regularAndStaticColumns()); - } - - // ONLY to be used for testing, to create a mock Memtable - @VisibleForTesting - public Memtable(TableMetadata metadata) - { - this.initialComparator = metadata.comparator; - this.cfs = null; - this.allocator = null; - this.columnsCollector = new ColumnsCollector(metadata.regularAndStaticColumns()); - } - - public MemtableAllocator getAllocator() - { - return allocator; - } - - public long getLiveDataSize() - { - return liveDataSize.get(); - } - - public long getOperations() - { - return currentOperations.get(); - } - - @VisibleForTesting - public void setDiscarding(OpOrder.Barrier writeBarrier, AtomicReference commitLogUpperBound) - { - assert this.writeBarrier == null; - this.commitLogUpperBound = commitLogUpperBound; - this.writeBarrier = writeBarrier; - allocator.setDiscarding(); - } - - void setDiscarded() - { - allocator.setDiscarded(); - } - - // decide if this memtable should take the write, or if it should go to the next memtable - public boolean accepts(OpOrder.Group opGroup, CommitLogPosition commitLogPosition) - { - // if the barrier hasn't been set yet, then this memtable is still taking ALL writes - OpOrder.Barrier barrier = this.writeBarrier; - if (barrier == null) - return true; - // if the barrier has been set, but is in the past, we are definitely destined for a future memtable - if (!barrier.isAfter(opGroup)) - return false; - // if we aren't durable we are directed only by the barrier - if (commitLogPosition == null) - return true; - while (true) - { - // otherwise we check if we are in the past/future wrt the CL boundary; - // if the boundary hasn't been finalised yet, we simply update it to the max of - // its current value and ours; if it HAS been finalised, we simply accept its judgement - // this permits us to coordinate a safe boundary, as the boundary choice is made - // atomically wrt our max() maintenance, so an operation cannot sneak into the past - CommitLogPosition currentLast = commitLogUpperBound.get(); - if (currentLast instanceof LastCommitLogPosition) - return currentLast.compareTo(commitLogPosition) >= 0; - if (currentLast != null && currentLast.compareTo(commitLogPosition) >= 0) - return true; - if (commitLogUpperBound.compareAndSet(currentLast, commitLogPosition)) - return true; - } - } - - public CommitLogPosition getCommitLogLowerBound() - { - return commitLogLowerBound.get(); - } - - public CommitLogPosition getCommitLogUpperBound() - { - return commitLogUpperBound.get(); - } - - public boolean isLive() - { - return allocator.isLive(); - } - - public boolean isClean() - { - return partitions.isEmpty(); - } - - public boolean mayContainDataBefore(CommitLogPosition position) - { - return approximateCommitLogLowerBound.compareTo(position) < 0; - } - - /** - * @return true if this memtable is expired. Expiration time is determined by CF's memtable_flush_period_in_ms. - */ - public boolean isExpired() - { - int period = cfs.metadata().params.memtableFlushPeriodInMs; - return period > 0 && (System.nanoTime() - creationNano >= TimeUnit.MILLISECONDS.toNanos(period)); - } - - /** - * Should only be called by ColumnFamilyStore.apply via Keyspace.apply, which supplies the appropriate - * OpOrdering. - * - * commitLogSegmentPosition should only be null if this is a secondary index, in which case it is *expected* to be null - */ - long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) - { - AtomicBTreePartition previous = partitions.get(update.partitionKey()); - - long initialSize = 0; - if (previous == null) - { - final DecoratedKey cloneKey = allocator.clone(update.partitionKey(), opGroup); - AtomicBTreePartition empty = new AtomicBTreePartition(cfs.metadata, cloneKey, allocator); - // We'll add the columns later. This avoids wasting works if we get beaten in the putIfAbsent - previous = partitions.putIfAbsent(cloneKey, empty); - if (previous == null) - { - previous = empty; - // allocate the row overhead after the fact; this saves over allocating and having to free after, but - // means we can overshoot our declared limit. - int overhead = (int) (cloneKey.getToken().getHeapSize() + ROW_OVERHEAD_HEAP_SIZE); - allocator.onHeap().allocate(overhead, opGroup); - initialSize = 8; - } - } - - long[] pair = previous.addAllWithSizeDelta(update, opGroup, indexer); - minTimestamp = Math.min(minTimestamp, previous.stats().minTimestamp); - liveDataSize.addAndGet(initialSize + pair[0]); - columnsCollector.update(update.columns()); - statsCollector.update(update.stats()); - currentOperations.addAndGet(update.operationCount()); - return pair[1]; - } - - public int partitionCount() - { - return partitions.size(); - } - - public List flushRunnables(LifecycleTransaction txn) - { - return createFlushRunnables(txn); - } - - private List createFlushRunnables(LifecycleTransaction txn) - { - DiskBoundaries diskBoundaries = cfs.getDiskBoundaries(); - List boundaries = diskBoundaries.positions; - List locations = diskBoundaries.directories; - if (boundaries == null) - return Collections.singletonList(new FlushRunnable(txn)); - - List runnables = new ArrayList<>(boundaries.size()); - PartitionPosition rangeStart = cfs.getPartitioner().getMinimumToken().minKeyBound(); - try - { - for (int i = 0; i < boundaries.size(); i++) - { - PartitionPosition t = boundaries.get(i); - runnables.add(new FlushRunnable(rangeStart, t, locations.get(i), txn)); - rangeStart = t; - } - return runnables; - } - catch (Throwable e) - { - throw Throwables.propagate(abortRunnables(runnables, e)); - } - } - - public Throwable abortRunnables(List runnables, Throwable t) - { - if (runnables != null) - for (FlushRunnable runnable : runnables) - t = runnable.writer.abort(t); - return t; - } - - public String toString() - { - return String.format("Memtable-%s@%s(%s serialized bytes, %s ops, %.0f%%/%.0f%% of on/off-heap limit)", - cfs.name, hashCode(), FBUtilities.prettyPrintMemory(liveDataSize.get()), currentOperations, - 100 * allocator.onHeap().ownershipRatio(), 100 * allocator.offHeap().ownershipRatio()); - } - - public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFilter columnFilter, final DataRange dataRange) - { - AbstractBounds keyRange = dataRange.keyRange(); - - boolean startIsMin = keyRange.left.isMinimum(); - boolean stopIsMin = keyRange.right.isMinimum(); - - boolean isBound = keyRange instanceof Bounds; - boolean includeStart = isBound || keyRange instanceof IncludingExcludingBounds; - boolean includeStop = isBound || keyRange instanceof Range; - Map subMap; - if (startIsMin) - subMap = stopIsMin ? partitions : partitions.headMap(keyRange.right, includeStop); - else - subMap = stopIsMin - ? partitions.tailMap(keyRange.left, includeStart) - : partitions.subMap(keyRange.left, includeStart, keyRange.right, includeStop); - - int minLocalDeletionTime = Integer.MAX_VALUE; - - // avoid iterating over the memtable if we purge all tombstones - if (cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones()) - minLocalDeletionTime = findMinLocalDeletionTime(subMap.entrySet().iterator()); - - final Iterator> iter = subMap.entrySet().iterator(); - - return new MemtableUnfilteredPartitionIterator(cfs, iter, minLocalDeletionTime, columnFilter, dataRange); - } - - private int findMinLocalDeletionTime(Iterator> iterator) - { - int minLocalDeletionTime = Integer.MAX_VALUE; - while (iterator.hasNext()) - { - Map.Entry entry = iterator.next(); - minLocalDeletionTime = Math.min(minLocalDeletionTime, entry.getValue().stats().minLocalDeletionTime); - } - return minLocalDeletionTime; - } - - public Partition getPartition(DecoratedKey key) - { - return partitions.get(key); - } - - public long getMinTimestamp() - { - return minTimestamp; - } - - /** - * For testing only. Give this memtable too big a size to make it always fail flushing. - */ - @VisibleForTesting - public void makeUnflushable() - { - liveDataSize.addAndGet((long) 1024 * 1024 * 1024 * 1024 * 1024); - } - - class FlushRunnable implements Callable - { - private final long estimatedSize; - private final ConcurrentNavigableMap toFlush; - - private final boolean isBatchLogTable; - private final SSTableMultiWriter writer; - - // keeping these to be able to log what we are actually flushing - private final PartitionPosition from; - private final PartitionPosition to; - - FlushRunnable(PartitionPosition from, PartitionPosition to, Directories.DataDirectory flushLocation, LifecycleTransaction txn) - { - this(partitions.subMap(from, to), flushLocation, from, to, txn); - } - - FlushRunnable(LifecycleTransaction txn) - { - this(partitions, null, null, null, txn); - } - - FlushRunnable(ConcurrentNavigableMap toFlush, Directories.DataDirectory flushLocation, PartitionPosition from, PartitionPosition to, LifecycleTransaction txn) - { - this.toFlush = toFlush; - this.from = from; - this.to = to; - long keySize = 0; - for (PartitionPosition key : toFlush.keySet()) - { - // make sure we don't write non-sensical keys - assert key instanceof DecoratedKey; - keySize += ((DecoratedKey) key).getKey().remaining(); - } - estimatedSize = (long) ((keySize // index entries - + keySize // keys in data file - + liveDataSize.get()) // data - * 1.2); // bloom filter and row index overhead - - this.isBatchLogTable = cfs.name.equals(SystemKeyspace.BATCHES) && cfs.keyspace.getName().equals(SchemaConstants.SYSTEM_KEYSPACE_NAME); - - if (flushLocation == null) - writer = createFlushWriter(txn, cfs.newSSTableDescriptor(getDirectories().getWriteableLocationAsFile(estimatedSize)), columnsCollector.get(), statsCollector.get()); - else - writer = createFlushWriter(txn, cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(flushLocation)), columnsCollector.get(), statsCollector.get()); - - } - - protected Directories getDirectories() - { - return cfs.getDirectories(); - } - - private void writeSortedContents() - { - logger.info("Writing {}, flushed range = ({}, {}]", Memtable.this.toString(), from, to); - - boolean trackContention = logger.isTraceEnabled(); - int heavilyContendedRowCount = 0; - // (we can't clear out the map as-we-go to free up memory, - // since the memtable is being used for queries in the "pending flush" category) - for (AtomicBTreePartition partition : toFlush.values()) - { - // Each batchlog partition is a separate entry in the log. And for an entry, we only do 2 - // operations: 1) we insert the entry and 2) we delete it. Further, BL data is strictly local, - // we don't need to preserve tombstones for repair. So if both operation are in this - // memtable (which will almost always be the case if there is no ongoing failure), we can - // just skip the entry (CASSANDRA-4667). - if (isBatchLogTable && !partition.partitionLevelDeletion().isLive() && partition.hasRows()) - continue; - - if (trackContention && partition.useLock()) - heavilyContendedRowCount++; - - if (!partition.isEmpty()) - { - try (UnfilteredRowIterator iter = partition.unfilteredIterator()) - { - writer.append(iter); - } - } - } - - long bytesFlushed = writer.getFilePointer(); - logger.info("Completed flushing {} ({}) for commitlog position {}", - writer.getFilename(), - FBUtilities.prettyPrintMemory(bytesFlushed), - commitLogUpperBound); - // Update the metrics - cfs.metric.bytesFlushed.inc(bytesFlushed); - - if (heavilyContendedRowCount > 0) - logger.trace("High update contention in {}/{} partitions of {} ", heavilyContendedRowCount, toFlush.size(), Memtable.this); - } - - public SSTableMultiWriter createFlushWriter(LifecycleTransaction txn, - Descriptor descriptor, - RegularAndStaticColumns columns, - EncodingStats stats) - { - MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata().comparator) - .commitLogIntervals(new IntervalSet<>(commitLogLowerBound.get(), commitLogUpperBound.get())); - - return cfs.createSSTableMultiWriter(descriptor, - toFlush.size(), - ActiveRepairService.UNREPAIRED_SSTABLE, - ActiveRepairService.NO_PENDING_REPAIR, - false, - sstableMetadataCollector, - new SerializationHeader(true, cfs.metadata(), columns, stats), txn); - } - - @Override - public SSTableMultiWriter call() - { - writeSortedContents(); - return writer; - } - } - - private static int estimateRowOverhead(final int count) - { - // calculate row overhead - try (final OpOrder.Group group = new OpOrder().start()) - { - int rowOverhead; - MemtableAllocator allocator = MEMORY_POOL.newAllocator(); - ConcurrentNavigableMap partitions = new ConcurrentSkipListMap<>(); - final Object val = new Object(); - for (int i = 0 ; i < count ; i++) - partitions.put(allocator.clone(new BufferDecoratedKey(new LongToken(i), ByteBufferUtil.EMPTY_BYTE_BUFFER), group), val); - double avgSize = ObjectSizes.measureDeep(partitions) / (double) count; - rowOverhead = (int) ((avgSize - Math.floor(avgSize)) < 0.05 ? Math.floor(avgSize) : Math.ceil(avgSize)); - rowOverhead -= ObjectSizes.measureDeep(new LongToken(0)); - rowOverhead += AtomicBTreePartition.EMPTY_SIZE; - rowOverhead += AbstractBTreePartition.HOLDER_UNSHARED_HEAP_SIZE; - allocator.setDiscarding(); - allocator.setDiscarded(); - return rowOverhead; - } - } - - public static class MemtableUnfilteredPartitionIterator extends AbstractUnfilteredPartitionIterator - { - private final ColumnFamilyStore cfs; - private final Iterator> iter; - private final int minLocalDeletionTime; - private final ColumnFilter columnFilter; - private final DataRange dataRange; - - public MemtableUnfilteredPartitionIterator(ColumnFamilyStore cfs, Iterator> iter, int minLocalDeletionTime, ColumnFilter columnFilter, DataRange dataRange) - { - this.cfs = cfs; - this.iter = iter; - this.minLocalDeletionTime = minLocalDeletionTime; - this.columnFilter = columnFilter; - this.dataRange = dataRange; - } - - public int getMinLocalDeletionTime() - { - return minLocalDeletionTime; - } - - public TableMetadata metadata() - { - return cfs.metadata(); - } - - public boolean hasNext() - { - return iter.hasNext(); - } - - public UnfilteredRowIterator next() - { - Map.Entry entry = iter.next(); - // Actual stored key should be true DecoratedKey - assert entry.getKey() instanceof DecoratedKey; - DecoratedKey key = (DecoratedKey)entry.getKey(); - ClusteringIndexFilter filter = dataRange.clusteringIndexFilter(key); - - return filter.getUnfilteredRowIterator(columnFilter, entry.getValue()); - } - } - - private static class ColumnsCollector - { - private final HashMap predefined = new HashMap<>(); - private final ConcurrentSkipListSet extra = new ConcurrentSkipListSet<>(); - ColumnsCollector(RegularAndStaticColumns columns) - { - for (ColumnMetadata def : columns.statics) - predefined.put(def, new AtomicBoolean()); - for (ColumnMetadata def : columns.regulars) - predefined.put(def, new AtomicBoolean()); - } - - public void update(RegularAndStaticColumns columns) - { - for (ColumnMetadata s : columns.statics) - update(s); - for (ColumnMetadata r : columns.regulars) - update(r); - } - - private void update(ColumnMetadata definition) - { - AtomicBoolean present = predefined.get(definition); - if (present != null) - { - if (!present.get()) - present.set(true); - } - else - { - extra.add(definition); - } - } - - public RegularAndStaticColumns get() - { - RegularAndStaticColumns.Builder builder = RegularAndStaticColumns.builder(); - for (Map.Entry e : predefined.entrySet()) - if (e.getValue().get()) - builder.add(e.getKey()); - return builder.addAll(extra).build(); - } - } - - private static class StatsCollector - { - private final AtomicReference stats = new AtomicReference<>(EncodingStats.NO_STATS); - - public void update(EncodingStats newStats) - { - while (true) - { - EncodingStats current = stats.get(); - EncodingStats updated = current.mergeWith(newStats); - if (stats.compareAndSet(current, updated)) - return; - } - } - - public EncodingStats get() - { - return stats.get(); - } - } -} diff --git a/src/java/org/apache/cassandra/db/MultiCBuilder.java b/src/java/org/apache/cassandra/db/MultiCBuilder.java index 0b5625b26f2a..b85b450c7a89 100644 --- a/src/java/org/apache/cassandra/db/MultiCBuilder.java +++ b/src/java/org/apache/cassandra/db/MultiCBuilder.java @@ -20,9 +20,15 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.NavigableSet; +import java.util.TreeSet; +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.db.marshal.CompositeType; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.btree.BTreeSet; @@ -129,6 +135,14 @@ public int remainingCount() return comparator.size() - size; } + /** + * Returns the current number of results when {@link #build()} is called + * + * @return the current number of build results + */ + public abstract int buildSize(); + + /** * Checks if the clusterings contains null elements. * @@ -165,6 +179,13 @@ public boolean hasMissingElements() */ public abstract NavigableSet> build(); + /** + * Builds the serialized partition keys. + * + * @return the serialized partition keys + */ + public abstract List buildSerializedPartitionKeys(); + /** * Builds the ClusteringBounds for slice restrictions. * @@ -252,6 +273,12 @@ public MultiCBuilder addAllElementsToAll(List> values) return addEachElementToAll(values.get(0)); } + @Override + public int buildSize() + { + return hasMissingElements ? 0 : 1; + } + public NavigableSet> build() { built = true; @@ -262,6 +289,23 @@ public NavigableSet> build() return BTreeSet.of(comparator, size == 0 ? Clustering.EMPTY : Clustering.make(elements)); } + @Override + public List buildSerializedPartitionKeys() + { + built = true; + + if (hasMissingElements) + return Collections.EMPTY_LIST; + + if (size == 0) + return ImmutableList.of(ByteBufferUtil.EMPTY_BYTE_BUFFER); + + if (size == 1) + return ImmutableList.of(elements[0]); + + return ImmutableList.of(CompositeType.build(ByteBufferAccessor.instance, elements)); + } + @Override public NavigableSet> buildBoundForSlice(boolean isStart, boolean isInclusive, @@ -397,6 +441,11 @@ public MultiCBuilder addAllElementsToAll(List> values) return this; } + public int buildSize() + { + return hasMissingElements ? 0 : elementsList.size(); + } + public NavigableSet> build() { built = true; @@ -418,6 +467,36 @@ public NavigableSet> build() return set.build(); } + @Override + public List buildSerializedPartitionKeys() + { + built = true; + + if (hasMissingElements) + return Collections.EMPTY_LIST; + + // Use a TreeSet here to remove duplicates and return the values in comparator sorted order + TreeSet set = comparator.size() == 1 ? new TreeSet<>(comparator.subtype(0)) + : new TreeSet<>(CompositeType.getInstance(comparator.subtypes())); + + for (int i = 0, m = elementsList.size(); i < m; i++) + { + List elements = elementsList.get(i); + set.add(comparator.size() == 1 ? elements.get(0) : toComposite(elements)); + } + return new ArrayList<>(set); + } + + private ByteBuffer toComposite(List elements) + { + ByteBuffer[] tmp = new ByteBuffer[elements.size()]; + for (int i = 0, m = elements.size(); i < m; i++) + { + tmp[i] = elements.get(i); + } + return CompositeType.build(ByteBufferAccessor.instance, tmp); + } + public NavigableSet> buildBoundForSlice(boolean isStart, boolean isInclusive, boolean isOtherBoundInclusive, diff --git a/src/java/org/apache/cassandra/db/MultiRangeReadCommand.java b/src/java/org/apache/cassandra/db/MultiRangeReadCommand.java new file mode 100644 index 000000000000..df980fed790c --- /dev/null +++ b/src/java/org/apache/cassandra/db/MultiRangeReadCommand.java @@ -0,0 +1,430 @@ +/* + * 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.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +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.PartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.metrics.TableMetrics; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.service.pager.PagingState; +import org.apache.cassandra.service.pager.QueryPager; +import org.apache.cassandra.service.reads.ReadCallback; +import org.apache.cassandra.transport.ProtocolVersion; + +/** + * Used by {@code EndpointGroupingCoordinator} to query all involved ranges on a given replica at once. + * + * Note: digest is not supported because each replica is responsible for different token ranges, there is no point on + * sending digest. + */ +public class MultiRangeReadCommand extends ReadCommand +{ + protected static final SelectionDeserializer selectionDeserializer = new Deserializer(); + + private final List dataRanges; + + private MultiRangeReadCommand(boolean isDigest, + int digestVersion, + boolean acceptsTransient, + TableMetadata metadata, + int nowInSec, + ColumnFilter columnFilter, + RowFilter rowFilter, + DataLimits limits, + List dataRanges, + Index.QueryPlan indexQueryPlan) + { + super(Kind.MULTI_RANGE, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, indexQueryPlan); + + assert dataRanges.size() > 0; + this.dataRanges = dataRanges; + } + + /** + * + * @param command current partition range command + * @param ranges token ranges to be queried on specific endpoint + * @param isRangeContinuation whether it's querying the first range in the batch + * @return multi-range read command for specific endpoint + */ + @VisibleForTesting + public static MultiRangeReadCommand create(PartitionRangeReadCommand command, List> ranges, boolean isRangeContinuation) + { + List dataRanges = new ArrayList<>(ranges.size()); + for (AbstractBounds range : ranges) + dataRanges.add(command.dataRange().forSubRange(range)); + + return new MultiRangeReadCommand(command.isDigestQuery(), + command.digestVersion(), + command.acceptsTransient(), + command.metadata(), + command.nowInSec(), + command.columnFilter(), + command.rowFilter(), + isRangeContinuation ? command.limits() : command.limits().withoutState(), + dataRanges, + command.indexQueryPlan()); + } + + /** + * @param subrangeHandlers handlers for all vnode ranges replicated in current endpoint. + * @return multi-range read command for specific endpoint + */ + public static MultiRangeReadCommand create(List> subrangeHandlers) + { + assert !subrangeHandlers.isEmpty(); + + PartitionRangeReadCommand command = (PartitionRangeReadCommand) subrangeHandlers.get(0).command(); + List dataRanges = new ArrayList<>(subrangeHandlers.size()); + for(ReadCallback handler : subrangeHandlers) + { + dataRanges.add(((PartitionRangeReadCommand) handler.command()).dataRange()); + } + + + return new MultiRangeReadCommand(command.isDigestQuery(), + command.digestVersion(), + command.acceptsTransient(), + command.metadata(), + command.nowInSec(), + command.columnFilter(), + command.rowFilter(), + command.limits(), + dataRanges, + command.indexQueryPlan()); + } + + /** + * @return all token ranges to be queried + */ + public List ranges() + { + return dataRanges; + } + + @Override + protected void serializeSelection(DataOutputPlus out, int version) throws IOException + { + int rangeCount = dataRanges.size(); + out.writeInt(rangeCount); + + for (DataRange range : dataRanges) + DataRange.serializer.serialize(range, out, version, metadata()); + } + + @Override + protected long selectionSerializedSize(int version) + { + int rangeCount = dataRanges.size(); + long size = TypeSizes.sizeof(rangeCount); + + for (DataRange range : dataRanges) + size += DataRange.serializer.serializedSize(range, version, metadata()); + + return size; + } + + @Override + public boolean isLimitedToOnePartition() + { + if (dataRanges.size() != 1) + return false; + + DataRange dataRange = dataRanges.get(0); + return dataRange.keyRange() instanceof Bounds + && dataRange.startKey().kind() == PartitionPosition.Kind.ROW_KEY + && dataRange.startKey().equals(dataRange.stopKey()); + } + + @Override + public boolean isRangeRequest() + { + return false; + } + + @Override + public ReadCommand withUpdatedLimit(DataLimits newLimits) + { + return new MultiRangeReadCommand(isDigestQuery(), + digestVersion(), + acceptsTransient(), + metadata(), + nowInSec(), + columnFilter(), + rowFilter(), + newLimits, + dataRanges, + indexQueryPlan()); + } + + @Override + public long getTimeout(TimeUnit unit) + { + return DatabaseDescriptor.getRangeRpcTimeout(unit); + } + + @Override + public ReadResponse createResponse(UnfilteredPartitionIterator iterator) + { + assert !isDigestQuery(); + return MultiRangeReadResponse.createDataResponse(iterator, this); + } + + @Override + public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key) + { + for (DataRange dataRange : ranges()) + { + if (dataRange.keyRange().contains(key)) + return dataRange.clusteringIndexFilter(key); + } + + throw new IllegalArgumentException(key + " is not in data ranges " + dataRanges.stream().map(r -> r.toString(metadata())).collect(Collectors.toList())); + } + + @Override + public ReadCommand copy() + { + return new MultiRangeReadCommand(isDigestQuery(), + digestVersion(), + acceptsTransient(), + metadata(), + nowInSec(), + columnFilter(), + rowFilter(), + limits(), + dataRanges, + indexQueryPlan()); + } + + @Override + protected ReadCommand copyAsTransientQuery() + { + return new MultiRangeReadCommand(false, + 0, + true, + metadata(), + nowInSec(), + columnFilter(), + rowFilter(), + limits(), + dataRanges, + indexQueryPlan()); + } + + @Override + protected ReadCommand copyAsDigestQuery() + { + throw new UnsupportedOperationException(); + } + + @Override + public UnfilteredPartitionIterator queryStorage(ColumnFamilyStore cfs, ReadExecutionController executionController) + { + return UnfilteredPartitionIterators.concat(dataRanges.stream() + .map(this::toPartitionRangeReadCommand) + .map(command -> command.queryStorage(cfs, executionController)) + .collect(Collectors.toList())); + } + + @Override + public UnfilteredPartitionIterator searchStorage(Index.Searcher searcher, ReadExecutionController controller) + { + if (indexQueryPlan.supportsMultiRangeReadCommand()) + { + // SAI supports fetching multiple ranges at once + return super.searchStorage(searcher, controller); + } + else + { + // search each subrange separately as they don't support MultiRangeReadCommand + return UnfilteredPartitionIterators.concat(dataRanges.stream() + .map(this::toPartitionRangeReadCommand) + .map(command -> command.searchStorage(searcher, controller)) + .collect(Collectors.toList())); + } + } + + private PartitionRangeReadCommand toPartitionRangeReadCommand(DataRange dataRange) + { + return PartitionRangeReadCommand.create(metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), dataRange, indexQueryPlan()); + } + + @Override + public boolean isReversed() + { + return ranges().get(0).isReversed(); + } + + @Override + protected void recordReadLatency(TableMetrics metric, long latencyNanos) + { + metric.rangeLatency.addNano(latencyNanos); + } + + @Override + protected void recordReadRequest(TableMetrics metric) + { + metric.rangeRequests.inc(); + } + + @Override + public Verb verb() + { + return Verb.MULTI_RANGE_REQ; + } + + @Override + protected void appendCQLWhereClause(StringBuilder sb) + { + if (ranges().size() == 1 && ranges().get(0).isUnrestricted() && rowFilter().isEmpty()) + return; + + sb.append(" WHERE "); + // We put the row filter first because the data range can end by "ORDER BY" + if (!rowFilter().isEmpty()) + { + sb.append(rowFilter()); + sb.append(" AND "); + } + + boolean isFirst = true; + for (int i = 0; i < ranges().size(); i++) + { + DataRange dataRange = ranges().get(i); + if (!dataRange.isUnrestricted()) + { + if (!isFirst) + sb.append(" AND "); + isFirst = false; + sb.append(dataRange.toCQLString(metadata())); + } + } + } + + @Override + public PartitionIterator execute(ConsistencyLevel consistency, QueryState queryState, long queryStartNanoTime) throws RequestExecutionException + { + // MultiRangeReadCommand should only be executed on the replica side + throw new UnsupportedOperationException(); + } + + @Override + public QueryPager getPager(PagingState pagingState, ProtocolVersion protocolVersion) + { + // MultiRangeReadCommand should only be executed at replica side" + throw new UnsupportedOperationException(); + } + + @Override + public boolean selectsKey(DecoratedKey key) + { + for (DataRange dataRange : ranges()) + { + if (!dataRange.contains(key)) + continue; + + return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().partitionKeyType); + } + + return false; + } + + @Override + public boolean selectsClustering(DecoratedKey key, Clustering clustering) + { + if (clustering == Clustering.STATIC_CLUSTERING) + return !columnFilter().fetchedColumns().statics.isEmpty(); + + for (DataRange dataRange : ranges()) + { + if (!dataRange.keyRange().contains(key) || !dataRange.clusteringIndexFilter(key).selects(clustering)) + continue; + + if (rowFilter().clusteringKeyRestrictionsAreSatisfiedBy(clustering)) + return true; + } + + return false; + } + + @Override + public boolean selectsFullPartition() + { + return metadata().isStaticCompactTable() || + (ranges().stream().allMatch(DataRange::selectsAllPartition) && !rowFilter().hasExpressionOnClusteringOrRegularColumns()); + } + + private static class Deserializer extends SelectionDeserializer + { + @Override + public ReadCommand deserialize(DataInputPlus in, + int version, + boolean isDigest, + int digestVersion, + boolean acceptsTransient, + TableMetadata metadata, + int nowInSec, + ColumnFilter columnFilter, + RowFilter rowFilter, + DataLimits limits, + Index.QueryPlan indexQueryPlan) + throws IOException + { + int rangeCount = in.readInt(); + + List ranges = new ArrayList<>(rangeCount); + for (int i = 0; i < rangeCount; i++) + ranges.add(DataRange.serializer.deserialize(in, version, metadata)); + + return new MultiRangeReadCommand(isDigest, + digestVersion, + acceptsTransient, + metadata, + nowInSec, + columnFilter, + rowFilter, + limits, + ranges, + indexQueryPlan); + } + } +} diff --git a/src/java/org/apache/cassandra/db/MultiRangeReadResponse.java b/src/java/org/apache/cassandra/db/MultiRangeReadResponse.java new file mode 100644 index 000000000000..29a7d1ef10b5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/MultiRangeReadResponse.java @@ -0,0 +1,411 @@ +/* + * 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.IOException; +import java.nio.ByteBuffer; +import java.util.NoSuchElementException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * It's used to store response of multi-range read request from a given endpoint, + * {@link ReadResponse} of subrange can be extracted via {@link #subrangeResponse(MultiRangeReadCommand, AbstractBounds)}; + */ +public abstract class MultiRangeReadResponse extends ReadResponse +{ + protected static final Logger logger = LoggerFactory.getLogger(MultiRangeReadResponse.class); + + public static final IVersionedSerializer serializer = new Serializer(); + + private MultiRangeReadResponse() + { + } + + /** + * @param data results of multiple ranges + * @param command current multi-range read command + * @return multi-range read response + */ + static ReadResponse createDataResponse(UnfilteredPartitionIterator data, MultiRangeReadCommand command) + { + return new LocalDataResponse(data, command); + } + + /** + * @param command current multi-range read command + * @param range target subrange + * @return response corresponding to the given range + */ + public abstract ReadResponse subrangeResponse(MultiRangeReadCommand command, AbstractBounds range); + + @Override + public ByteBuffer digest(ReadCommand command) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isDigestResponse() + { + return false; + } + + @Override + public ByteBuffer repairedDataDigest() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isRepairedDigestConclusive() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean mayIncludeRepairedDigest() + { + throw new UnsupportedOperationException(); + } + + @Override + public String toDebugString(ReadCommand command, DecoratedKey key) + { + throw new UnsupportedOperationException(); + } + + /** + * A local response that is not meant to be serialized or used for caching remote endpoint's multi-range response. + */ + private static class LocalResponse extends MultiRangeReadResponse + { + private final RangeBoundPartitionIterator iterator; + + LocalResponse(UnfilteredPartitionIterator response) + { + this.iterator = new RangeBoundPartitionIterator(response); + } + + @Override + public UnfilteredPartitionIterator makeIterator(ReadCommand command) + { + throw new UnsupportedOperationException(); + } + + @Override + public ReadResponse subrangeResponse(MultiRangeReadCommand command, AbstractBounds range) + { + // deliver already cached content without deserialization. + return new LocalSubrangeResponse(iterator, range); + } + + class RangeBoundPartitionIterator + { + private final UnfilteredPartitionIterator iterator; + private UnfilteredRowIterator next = null; + + RangeBoundPartitionIterator(UnfilteredPartitionIterator iterator) + { + this.iterator = iterator; + } + + public boolean hasNext(AbstractBounds range) + { + if (next != null) + return range.contains(next.partitionKey()); + + if (iterator.hasNext()) + { + next = iterator.next(); + if (range.contains(next.partitionKey())) + return true; + } + return false; + } + + public UnfilteredRowIterator next() + { + if (next != null) + { + UnfilteredRowIterator result = next; + next = null; + return result; + } + throw new NoSuchElementException(); + } + } + } + + private static class LocalSubrangeResponse extends ReadResponse + { + private final LocalResponse.RangeBoundPartitionIterator iterator; + private final AbstractBounds range; + + LocalSubrangeResponse(LocalResponse.RangeBoundPartitionIterator iterator, AbstractBounds range) + { + this.iterator = iterator; + this.range = range; + } + + @Override + public UnfilteredPartitionIterator makeIterator(ReadCommand command) + { + return new AbstractUnfilteredPartitionIterator() + { + @Override + public TableMetadata metadata() + { + return command.metadata(); + } + + @Override + public boolean hasNext() + { + return iterator.hasNext(range); + } + + @Override + public UnfilteredRowIterator next() + { + return iterator.next(); + } + }; + } + + @Override + public ByteBuffer digest(ReadCommand command) + { + throw new UnsupportedOperationException(); + } + + @Override + public ByteBuffer repairedDataDigest() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isRepairedDigestConclusive() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean mayIncludeRepairedDigest() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isDigestResponse() + { + return false; + } + } + + /** + * A local response that needs to be serialized, i.e. sent to another node. The iterator + * is serialized by the build method and can be closed as soon as this response has been created. + */ + private static class LocalDataResponse extends DataResponse + { + private LocalDataResponse(UnfilteredPartitionIterator iterator, MultiRangeReadCommand command) + { + super(build(iterator, command.columnFilter()), MessagingService.current_version, DeserializationHelper.Flag.FROM_REMOTE); + } + + private static ByteBuffer build(UnfilteredPartitionIterator iterator, ColumnFilter selection) + { + try (DataOutputBuffer buffer = new DataOutputBuffer()) + { + UnfilteredPartitionIterators.serializerForIntraNode().serialize(iterator, selection, buffer, MessagingService.current_version); + return buffer.buffer(); + } + catch (IOException e) + { + // We're serializing in memory so this shouldn't happen + throw new RuntimeException(e); + } + } + } + + /** + * A response received from a remove node. We keep the response serialized in the byte buffer. + */ + private static class RemoteDataResponse extends DataResponse + { + RemoteDataResponse(ByteBuffer data, + int dataSerializationVersion) + { + super(data, dataSerializationVersion, DeserializationHelper.Flag.FROM_REMOTE); + } + } + + /** + * The command base class for local or remote responses that stay serialized in a byte buffer, + * the data. + */ + static abstract class DataResponse extends MultiRangeReadResponse + { + // The response, serialized in the current messaging version + private final ByteBuffer data; + private final int dataSerializationVersion; + private final DeserializationHelper.Flag flag; + + private MultiRangeReadResponse.LocalResponse cached; + + DataResponse(ByteBuffer data, + int dataSerializationVersion, + DeserializationHelper.Flag flag) + { + this.data = data; + this.dataSerializationVersion = dataSerializationVersion; + this.flag = flag; + } + + public UnfilteredPartitionIterator makeIterator(ReadCommand command) + { + try (DataInputBuffer in = new DataInputBuffer(data, true)) + { + // Note that the command parameter shadows the 'command' field and this is intended because + // the later can be null (for RemoteDataResponse as those are created in the serializers and + // those don't have easy access to the command). This is also why we need the command as parameter here. + return UnfilteredPartitionIterators.serializerForIntraNode().deserialize(in, + dataSerializationVersion, + command.metadata(), + command.columnFilter(), + flag); + } + catch (IOException e) + { + // We're deserializing in memory so this shouldn't happen + throw new RuntimeException(e); + } + } + + public ByteBuffer repairedDataDigest() + { + return ByteBufferUtil.EMPTY_BYTE_BUFFER; + } + + @Override + public boolean isRepairedDigestConclusive() + { + return true; + } + + @Override + public boolean mayIncludeRepairedDigest() + { + return dataSerializationVersion >= MessagingService.VERSION_40; + } + + @Override + public ReadResponse subrangeResponse(MultiRangeReadCommand command, AbstractBounds range) + { + if (cached == null) + { + try (DataInputBuffer in = new DataInputBuffer(data, true)) + { + @SuppressWarnings("resource") // The close operation is a noop for a deserialized UPI + UnfilteredPartitionIterator iterator = UnfilteredPartitionIterators.serializerForIntraNode() + .deserialize(in, + dataSerializationVersion, + command.metadata(), + command.columnFilter(), + flag); + cached = new LocalResponse(iterator); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } + + return cached.subrangeResponse(command, range); + } + } + + /** + * A copy of {@code ReadResponse.Serializer} that doesn't support a digest response + */ + private static class Serializer implements IVersionedSerializer + { + public void serialize(ReadResponse response, DataOutputPlus out, int version) throws IOException + { + ByteBuffer digest = ByteBufferUtil.EMPTY_BYTE_BUFFER; + ByteBufferUtil.writeWithVIntLength(digest, out); + if (version >= MessagingService.VERSION_40) + { + ByteBufferUtil.writeWithVIntLength(response.repairedDataDigest(), out); + out.writeBoolean(response.isRepairedDigestConclusive()); + } + ByteBuffer data = ((DataResponse)response).data; + ByteBufferUtil.writeWithVIntLength(data, out); + } + + public ReadResponse deserialize(DataInputPlus in, int version) throws IOException + { + ByteBuffer digest = ByteBufferUtil.readWithVIntLength(in); + assert !digest.hasRemaining(); + + if (version >= MessagingService.VERSION_40) + { + ByteBufferUtil.readWithVIntLength(in); + in.readBoolean(); + } + ByteBuffer data = ByteBufferUtil.readWithVIntLength(in); + return new RemoteDataResponse(data, version); + } + + public long serializedSize(ReadResponse response, int version) + { + ByteBuffer digest = ByteBufferUtil.EMPTY_BYTE_BUFFER; + long size = ByteBufferUtil.serializedSizeWithVIntLength(digest); + + if (version >= MessagingService.VERSION_40) + { + size += ByteBufferUtil.serializedSizeWithVIntLength(response.repairedDataDigest()); + size += 1; + } + assert version >= MessagingService.VERSION_30; + ByteBuffer data = ((DataResponse)response).data; + size += ByteBufferUtil.serializedSizeWithVIntLength(data); + return size; + } + } +} diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java index 8a1ffc123c54..d2d817d02928 100644 --- a/src/java/org/apache/cassandra/db/Mutation.java +++ b/src/java/org/apache/cassandra/db/Mutation.java @@ -39,9 +39,11 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.ByteBufferUtil; +import static org.apache.cassandra.net.MessagingService.VERSION_SG_10; import static org.apache.cassandra.net.MessagingService.VERSION_30; import static org.apache.cassandra.net.MessagingService.VERSION_3014; import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.MessagingService.VERSION_41; import static org.apache.cassandra.utils.MonotonicClock.approxTime; public class Mutation implements IMutation @@ -282,6 +284,8 @@ public String toString(boolean shallow) private int serializedSize30; private int serializedSize3014; private int serializedSize40; + private int serializedSize41; + private int serializedSizeSG10; public int serializedSize(int version) { @@ -299,6 +303,14 @@ public int serializedSize(int version) if (serializedSize40 == 0) serializedSize40 = (int) serializer.serializedSize(this, VERSION_40); return serializedSize40; + case VERSION_41: + if (serializedSize41 == 0) + serializedSize41 = (int) serializer.serializedSize(this, VERSION_41); + return serializedSize41; + case VERSION_SG_10: + if (serializedSizeSG10 == 0) + serializedSizeSG10 = (int) serializer.serializedSize(this, VERSION_SG_10); + return serializedSizeSG10; default: throw new IllegalStateException("Unknown serialization version: " + version); } diff --git a/src/java/org/apache/cassandra/db/NativeClustering.java b/src/java/org/apache/cassandra/db/NativeClustering.java index 0e4c19db17ef..8ac81520bf69 100644 --- a/src/java/org/apache/cassandra/db/NativeClustering.java +++ b/src/java/org/apache/cassandra/db/NativeClustering.java @@ -25,6 +25,7 @@ import org.apache.cassandra.db.marshal.ValueAccessor; import org.apache.cassandra.utils.ObjectSizes; 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; @@ -36,11 +37,6 @@ public class NativeClustering implements Clustering private NativeClustering() { peer = 0; } - public ClusteringPrefix minimize() - { - return this; - } - public NativeClustering(NativeAllocator allocator, OpOrder.Group writeOp, Clustering clustering) { int count = clustering.size(); @@ -157,4 +153,19 @@ public final boolean equals(Object o) { return ClusteringPrefix.equals(this, o); } + + public ClusteringPrefix retainable() + { + assert kind() == Kind.CLUSTERING; // tombstones are never stored natively + + // always extract + ByteBuffer[] values = new ByteBuffer[size()]; + for (int i = 0; i < values.length; ++i) + { + ByteBuffer value = get(i); + values[i] = value != null ? HeapAllocator.instance.clone(value) : null; + } + + return accessor().factory().clustering(values); + } } diff --git a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java index add52189776c..461a25a4f54a 100644 --- a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java +++ b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java @@ -20,7 +20,11 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; +import net.nicoulaj.compilecommand.annotations.Inline; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.memory.MemoryUtil; import org.apache.cassandra.utils.memory.NativeAllocator; @@ -41,8 +45,72 @@ public NativeDecoratedKey(Token token, NativeAllocator allocator, OpOrder.Group MemoryUtil.setBytes(peer + 4, key); } + public NativeDecoratedKey(Token token, NativeAllocator allocator, OpOrder.Group writeOp, byte[] keyBytes) + { + super(token); + assert keyBytes != null; + + int size = keyBytes.length; + this.peer = allocator.allocate(4 + size, writeOp); + MemoryUtil.setInt(peer, size); + MemoryUtil.setBytes(peer + 4, keyBytes, 0, size); + } + + @Inline + int length() + { + return MemoryUtil.getInt(peer); + } + + @Inline + long address() + { + return this.peer + 4; + } + + @Override public ByteBuffer getKey() { return MemoryUtil.getByteBuffer(peer + 4, MemoryUtil.getInt(peer), ByteOrder.BIG_ENDIAN); } + + @Override + public int getKeyLength() + { + return MemoryUtil.getInt(peer); + } + + @Override + protected ByteSource keyComparableBytes(Version version) + { + return ByteSource.of(address(), length(), version); + } + + /** + * A factory method that translates the given byte-comparable representation to a {@link NativeDecoratedKey} + * instance. If the given byte comparable doesn't represent the encoding of a native decorated key, anything from a + * wide variety of throwables may be thrown (e.g. {@link AssertionError}, {@link IndexOutOfBoundsException}, + * {@link IllegalStateException}, etc.). + * + * @param byteComparable A byte-comparable representation (presumably of a {@link NativeDecoratedKey} instance). + * @param version The encoding version used for the given byte comparable. + * @param partitioner The partitioner of the encoded decorated key. Needed in order to correctly decode the token + * bytes of the key. + * @param allocator The native allocator needed to copy the key contents to off-heap memory. + * + * @return A new {@link NativeDecoratedKey} instance, corresponding to the given byte-comparable representation. If + * we were to call {@link #asComparableBytes(Version)} on the returned object, we should get a {@link ByteSource} + * equal to the one of the input byte comparable. + */ + public static NativeDecoratedKey fromByteComparable(ByteComparable byteComparable, + ByteComparable.Version version, + IPartitioner partitioner, + NativeAllocator allocator, + OpOrder.Group opGroup) + { + return DecoratedKey.fromByteComparable(byteComparable, + version, + partitioner, + (token, keyBytes) -> new NativeDecoratedKey(token, allocator, opGroup, keyBytes)); + } } diff --git a/src/java/org/apache/cassandra/db/PartitionPosition.java b/src/java/org/apache/cassandra/db/PartitionPosition.java index 3b45c6c0e2eb..661dc77d0f26 100644 --- a/src/java/org/apache/cassandra/db/PartitionPosition.java +++ b/src/java/org/apache/cassandra/db/PartitionPosition.java @@ -24,8 +24,10 @@ import org.apache.cassandra.dht.*; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; -public interface PartitionPosition extends RingPosition +public interface PartitionPosition extends RingPosition, ByteComparable { public static enum Kind { @@ -54,6 +56,20 @@ public static PartitionPosition get(ByteBuffer key, IPartitioner p) public Kind kind(); public boolean isMinimum(); + /** + * Produce a prefix-free byte-comparable representation of the key, i.e. such a sequence of bytes that any pair x, y + * of valid positions (with the same key column types and partitioner), + * x.compareTo(y) == compareLexicographicallyUnsigned(x.asComparableBytes(), y.asComparableBytes()) + * and + * x.asComparableBytes() is not a prefix of y.asComparableBytes() + * + * We use a two-component tuple for decorated keys, and a one-component tuple for key bounds, where the terminator + * byte is chosen to yield the correct comparison result. No decorated key can be a prefix of another (per the tuple + * encoding), and no key bound can be a prefix of one because it uses a terminator byte that is different from the + * tuple separator. + */ + public abstract ByteSource asComparableBytes(Version version); + public static class RowPositionSerializer implements IPartitionerDependentSerializer { /* diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java index 82b6e8a6e8de..ad2f745f285b 100644 --- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java +++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java @@ -22,7 +22,7 @@ import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.net.MessageFlag; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.config.DatabaseDescriptor; @@ -41,9 +41,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.metrics.TableMetrics; -import org.apache.cassandra.net.Message; -import org.apache.cassandra.schema.IndexMetadata; -import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageProxy; import org.apache.cassandra.tracing.Tracing; @@ -65,12 +63,32 @@ private PartitionRangeReadCommand(boolean isDigest, RowFilter rowFilter, DataLimits limits, DataRange dataRange, - IndexMetadata index) + Index.QueryPlan indexQueryPlan) { - super(Kind.PARTITION_RANGE, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, index); + super(Kind.PARTITION_RANGE, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, indexQueryPlan); this.dataRange = dataRange; } + public static PartitionRangeReadCommand create(TableMetadata metadata, + int nowInSec, + ColumnFilter columnFilter, + RowFilter rowFilter, + DataLimits limits, + DataRange dataRange, + Index.QueryPlan indexQueryPlan) + { + return new PartitionRangeReadCommand(false, + 0, + false, + metadata, + nowInSec, + columnFilter, + rowFilter, + limits, + dataRange, + indexQueryPlan); + } + public static PartitionRangeReadCommand create(TableMetadata metadata, int nowInSec, ColumnFilter columnFilter, @@ -87,7 +105,7 @@ public static PartitionRangeReadCommand create(TableMetadata metadata, rowFilter, limits, dataRange, - findIndex(metadata, rowFilter)); + findIndexQueryPlan(metadata, rowFilter)); } /** @@ -158,7 +176,7 @@ public PartitionRangeReadCommand forSubRange(AbstractBounds r rowFilter(), isRangeContinuation ? limits() : limits().withoutState(), dataRange().forSubRange(range), - indexMetadata()); + indexQueryPlan()); } public PartitionRangeReadCommand copy() @@ -172,7 +190,7 @@ public PartitionRangeReadCommand copy() rowFilter(), limits(), dataRange(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -187,7 +205,7 @@ protected PartitionRangeReadCommand copyAsDigestQuery() rowFilter(), limits(), dataRange(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -202,7 +220,7 @@ protected PartitionRangeReadCommand copyAsTransientQuery() rowFilter(), limits(), dataRange(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -217,7 +235,7 @@ public PartitionRangeReadCommand withUpdatedLimit(DataLimits newLimits) rowFilter(), newLimits, dataRange(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -232,7 +250,7 @@ public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLim rowFilter(), newLimits, newDataRange, - indexMetadata()); + indexQueryPlan()); } public long getTimeout(TimeUnit unit) @@ -245,16 +263,21 @@ public boolean isReversed() return dataRange.isReversed(); } - public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException + public PartitionIterator execute(ConsistencyLevel consistency, QueryState queryState, long queryStartNanoTime) throws RequestExecutionException { return StorageProxy.getRangeSlice(this, consistency, queryStartNanoTime); } - protected void recordLatency(TableMetrics metric, long latencyNanos) + protected void recordReadLatency(TableMetrics metric, long latencyNanos) { metric.rangeLatency.addNano(latencyNanos); } + protected void recordReadRequest(TableMetrics metric) + { + metric.rangeRequests.inc(); + } + @VisibleForTesting public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController) { @@ -265,11 +288,15 @@ public UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, Rea InputCollector inputCollector = iteratorsForRange(view); try { + // avoid iterating over the memtable if we purge all tombstones + boolean useMinLocalDeletionTime = cfs.onlyPurgeRepairedTombstones(); + for (Memtable memtable : view.memtables) { @SuppressWarnings("resource") // We close on exception and on closing the result returned by this method Memtable.MemtableUnfilteredPartitionIterator iter = memtable.makePartitionIterator(columnFilter(), dataRange()); - oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, iter.getMinLocalDeletionTime()); + if (useMinLocalDeletionTime) + oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, iter.getMinLocalDeletionTime()); inputCollector.addMemtableIterator(RTBoundValidator.validate(iter, RTBoundValidator.Stage.MEMTABLE, false)); } @@ -383,9 +410,8 @@ protected void appendCQLWhereClause(StringBuilder sb) */ public PartitionIterator postReconciliationProcessing(PartitionIterator result) { - ColumnFamilyStore cfs = Keyspace.open(metadata().keyspace).getColumnFamilyStore(metadata().name); - Index index = getIndex(cfs); - return index == null ? result : index.postProcessorFor(this).apply(result, this); + Index.QueryPlan queryPlan = indexQueryPlan(); + return queryPlan == null ? result : queryPlan.postProcessor().apply(result); } @Override @@ -439,11 +465,11 @@ public ReadCommand deserialize(DataInputPlus in, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, - IndexMetadata index) + Index.QueryPlan indexQueryPlan) throws IOException { DataRange range = DataRange.serializer.deserialize(in, version, metadata); - return new PartitionRangeReadCommand(isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, range, index); + return new PartitionRangeReadCommand(isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, range, indexQueryPlan); } } } diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 7b889d188a18..5446ad7174c7 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -34,8 +34,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.*; import org.apache.cassandra.db.filter.*; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.guardrails.Guardrail; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.net.MessageFlag; import org.apache.cassandra.net.Verb; import org.apache.cassandra.db.partitions.*; @@ -47,8 +49,6 @@ import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.exceptions.UnknownIndexException; import org.apache.cassandra.index.Index; -import org.apache.cassandra.index.IndexNotAvailableException; -import org.apache.cassandra.index.IndexRegistry; import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataInputPlus; @@ -63,7 +63,6 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.SchemaProvider; import org.apache.cassandra.service.ActiveRepairService; -import org.apache.cassandra.service.ClientWarn; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.FBUtilities; @@ -99,7 +98,7 @@ public abstract class ReadCommand extends AbstractReadQuery int oldestUnrepairedTombstone = Integer.MAX_VALUE; @Nullable - private final IndexMetadata index; + protected final Index.QueryPlan indexQueryPlan; protected static abstract class SelectionDeserializer { @@ -113,13 +112,14 @@ public abstract ReadCommand deserialize(DataInputPlus in, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, - IndexMetadata index) throws IOException; + Index.QueryPlan indexQueryPlan) throws IOException; } protected enum Kind { SINGLE_PARTITION (SinglePartitionReadCommand.selectionDeserializer), - PARTITION_RANGE (PartitionRangeReadCommand.selectionDeserializer); + PARTITION_RANGE (PartitionRangeReadCommand.selectionDeserializer), + MULTI_RANGE (MultiRangeReadCommand.selectionDeserializer); private final SelectionDeserializer selectionDeserializer; @@ -138,7 +138,7 @@ protected ReadCommand(Kind kind, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, - IndexMetadata index) + Index.QueryPlan indexQueryPlan) { super(metadata, nowInSec, columnFilter, rowFilter, limits); if (acceptsTransient && isDigestQuery) @@ -148,7 +148,7 @@ protected ReadCommand(Kind kind, this.isDigestQuery = isDigestQuery; this.digestVersion = digestVersion; this.acceptsTransient = acceptsTransient; - this.index = index; + this.indexQueryPlan = indexQueryPlan; } protected abstract void serializeSelection(DataOutputPlus out, int version) throws IOException; @@ -281,14 +281,20 @@ public boolean isRepairedDataDigestConclusive() } /** - * Index (metadata) chosen for this query. Can be null. + * Index query plan chosen for this query. Can be null. * - * @return index (metadata) chosen for this query + * @return index query plan chosen for this query */ @Nullable - public IndexMetadata indexMetadata() + public Index.QueryPlan indexQueryPlan() { - return index; + return indexQueryPlan; + } + + @VisibleForTesting + public Index.Searcher indexSearcher() + { + return indexQueryPlan == null ? null : indexQueryPlan.searcherFor(this); } /** @@ -382,32 +388,38 @@ public ReadResponse createResponse(UnfilteredPartitionIterator iterator) : ReadResponse.createDataResponse(iterator, this); } + public DataLimits.Counter createLimitedCounter(boolean assumeLiveData) + { + return limits().newCounter(nowInSec(), assumeLiveData, selectsFullPartition(), metadata().enforceStrictLiveness()).onlyCount(); + } + + public DataLimits.Counter createUnlimitedCounter(boolean assumeLiveData) + { + return DataLimits.NONE.newCounter(nowInSec(), assumeLiveData, selectsFullPartition(), metadata().enforceStrictLiveness()); + } + long indexSerializedSize(int version) { - return null != index - ? IndexMetadata.serializer.serializedSize(index, version) + return null != indexQueryPlan + ? IndexMetadata.serializer.serializedSize(indexQueryPlan.getFirst().getIndexMetadata(), version) : 0; } public Index getIndex(ColumnFamilyStore cfs) { - return null != index - ? cfs.indexManager.getIndex(index) + return null != indexQueryPlan + ? indexQueryPlan.getFirst() : null; } - static IndexMetadata findIndex(TableMetadata table, RowFilter rowFilter) + static Index.QueryPlan findIndexQueryPlan(TableMetadata table, RowFilter rowFilter) { if (table.indexes.isEmpty() || rowFilter.isEmpty()) return null; ColumnFamilyStore cfs = Keyspace.openAndGetStore(table); - Index index = cfs.indexManager.getBestIndexFor(rowFilter); - - return null != index - ? index.getIndexMetadata() - : null; + return cfs.indexManager.getBestIndexQueryPlanFor(rowFilter); } /** @@ -418,8 +430,8 @@ static IndexMetadata findIndex(TableMetadata table, RowFilter rowFilter) */ public void maybeValidateIndex() { - if (null != index) - IndexRegistry.obtain(metadata()).getIndex(index).validate(this); + if (null != indexQueryPlan) + indexQueryPlan.validate(this); } /** @@ -436,15 +448,13 @@ public UnfilteredPartitionIterator executeLocally(ReadExecutionController execut long startTimeNanos = System.nanoTime(); ColumnFamilyStore cfs = Keyspace.openAndGetStore(metadata()); - Index index = getIndex(cfs); Index.Searcher searcher = null; - if (index != null) + if (indexQueryPlan != null) { - if (!cfs.indexManager.isIndexQueryable(index)) - throw new IndexNotAvailableException(index); - - searcher = index.searcherFor(this); + cfs.indexManager.checkQueryability(indexQueryPlan); + searcher = indexSearcher(); + Index index = indexQueryPlan.getFirst(); Tracing.trace("Executing read on {}.{} using index {}", cfs.metadata.keyspace, cfs.metadata.name, index.getIndexMetadata().name); } @@ -457,7 +467,8 @@ public UnfilteredPartitionIterator executeLocally(ReadExecutionController execut repairedDataInfo = new RepairedDataInfo(repairedReadCount); } - UnfilteredPartitionIterator iterator = (null == searcher) ? queryStorage(cfs, executionController) : searcher.search(executionController); + UnfilteredPartitionIterator iterator = (null == searcher) ? queryStorage(cfs, executionController) + : searchStorage(searcher, executionController); iterator = RTBoundValidator.validate(iterator, Stage.MERGED, false); try @@ -468,7 +479,7 @@ public UnfilteredPartitionIterator executeLocally(ReadExecutionController execut // If we've used a 2ndary index, we know the result already satisfy the primary expression used, so // no point in checking it again. - RowFilter filter = (null == searcher) ? rowFilter() : index.getPostIndexQueryFilter(rowFilter()); + RowFilter filter = (null == searcher) ? rowFilter() : indexQueryPlan.postIndexQueryFilter(); /* * TODO: We'll currently do filtering by the rowFilter here because it's convenient. However, @@ -507,13 +518,47 @@ public UnfilteredPartitionIterator executeLocally(ReadExecutionController execut } } - protected abstract void recordLatency(TableMetrics metric, long latencyNanos); + public UnfilteredPartitionIterator searchStorage(Index.Searcher searcher, ReadExecutionController executionController) + { + return searcher.search(executionController); + } + + protected abstract void recordReadRequest(TableMetrics metric); + protected abstract void recordReadLatency(TableMetrics metric, long latencyNanos); + + /** + * Allow to post-process the result of the query after it has been reconciled on the coordinator + * but before it is passed to the CQL layer to return the ResultSet. + * + * See CASSANDRA-8717 for why this exists. + */ + public PartitionIterator postReconciliationProcessing(PartitionIterator result) + { + return indexQueryPlan == null ? result : indexQueryPlan.postProcessor().apply(result); + } + + @Override + public PartitionIterator executeInternal(ReadExecutionController controller) + { + return postReconciliationProcessing(UnfilteredPartitionIterators.filter(executeLocally(controller), nowInSec())); + } public ReadExecutionController executionController() { return ReadExecutionController.forCommand(this); } + /** + * Whether tombstone guardrail ({@link Guardrails#scannedTombstones} should be respected for this query. + * + * @return {@code true} if the tombstone thresholds should be respected for the query. If {@code false}, no + * tombstone warning will ever be logged, and the query will never fail due to tombstones. + */ + protected boolean shouldRespectTombstoneThresholds() + { + return !SchemaConstants.isLocalSystemKeyspace(ReadCommand.this.metadata().keyspace); + } + /** * Wraps the provided iterator so that metrics on what is scanned by the command are recorded. * This also log warning/trow TombstoneOverwhelmingException if appropriate. @@ -522,17 +567,26 @@ private UnfilteredPartitionIterator withMetricsRecording(UnfilteredPartitionIter { class MetricRecording extends Transformation { - private final int failureThreshold = DatabaseDescriptor.getTombstoneFailureThreshold(); - private final int warningThreshold = DatabaseDescriptor.getTombstoneWarnThreshold(); - - private final boolean respectTombstoneThresholds = !SchemaConstants.isLocalSystemKeyspace(ReadCommand.this.metadata().keyspace); private final boolean enforceStrictLiveness = metadata().enforceStrictLiveness(); private int liveRows = 0; - private int tombstones = 0; + private final Guardrail.Threshold.GuardedCounter tombstones = createTombstoneCounter(); private DecoratedKey currentKey; + private Guardrail.Threshold.GuardedCounter createTombstoneCounter() + { + Guardrail.Threshold guardrail = shouldRespectTombstoneThresholds() + ? Guardrails.scannedTombstones + : Guardrail.Threshold.NEVER_TRIGGERED; + return guardrail.newCounter(ReadCommand.this::toCQLString, true, null); + } + + private MetricRecording() + { + recordReadRequest(metric); + } + @Override public UnfilteredRowIterator applyToPartition(UnfilteredRowIterator iter) { @@ -581,42 +635,33 @@ public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker) private void countTombstone(ClusteringPrefix clustering) { - ++tombstones; - if (tombstones > failureThreshold && respectTombstoneThresholds) + try + { + tombstones.add(1); + } + catch (InvalidRequestException e) { - String query = ReadCommand.this.toCQLString(); - Tracing.trace("Scanned over {} tombstones for query {}; query aborted (see tombstone_failure_threshold)", failureThreshold, query); metric.tombstoneFailures.inc(); - throw new TombstoneOverwhelmingException(tombstones, query, ReadCommand.this.metadata(), currentKey, clustering); + throw new TombstoneOverwhelmingException(tombstones.get(), + ReadCommand.this.toCQLString(), + ReadCommand.this.metadata(), + currentKey, + clustering); } } @Override public void onClose() { - recordLatency(metric, System.nanoTime() - startTimeNanos); + recordReadLatency(metric, System.nanoTime() - startTimeNanos); - metric.tombstoneScannedHistogram.update(tombstones); + metric.tombstoneScannedHistogram.update(tombstones.get()); metric.liveScannedHistogram.update(liveRows); - boolean warnTombstones = tombstones > warningThreshold && respectTombstoneThresholds; - if (warnTombstones) - { - String msg = String.format( - "Read %d live rows and %d tombstone cells for query %1.512s; token %s (see tombstone_warn_threshold)", - liveRows, tombstones, ReadCommand.this.toCQLString(), currentKey.getToken()); - ClientWarn.instance.warn(msg); - if (tombstones < failureThreshold) - { - metric.tombstoneWarnings.inc(); - } - - logger.warn(msg); - } + if (tombstones.checkAndTriggerWarning()) + metric.tombstoneWarnings.inc(); - Tracing.trace("Read {} live rows and {} tombstone cells{}", - liveRows, tombstones, - (warnTombstones ? " (see tombstone_warn_threshold)" : "")); + Tracing.trace("Read {} live rows and {} tombstone ones", liveRows, tombstones.get()); } } @@ -704,7 +749,7 @@ class WithoutPurgeableTombstones extends PurgeFunction public WithoutPurgeableTombstones() { super(nowInSec(), cfs.gcBefore(nowInSec()), oldestUnrepairedTombstone(), - cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones(), + cfs.onlyPurgeRepairedTombstones(), iterator.metadata().enforceStrictLiveness()); } @@ -967,7 +1012,7 @@ public void serialize(ReadCommand command, DataOutputPlus out, int version) thro out.writeByte(command.kind.ordinal()); out.writeByte( digestFlag(command.isDigestQuery()) - | indexFlag(null != command.indexMetadata()) + | indexFlag(null != command.indexQueryPlan()) | acceptsTransientFlag(command.acceptsTransient()) ); if (command.isDigestQuery()) @@ -977,8 +1022,8 @@ public void serialize(ReadCommand command, DataOutputPlus out, int version) thro ColumnFilter.serializer.serialize(command.columnFilter(), out, version); RowFilter.serializer.serialize(command.rowFilter(), out, version); DataLimits.serializer.serialize(command.limits(), out, version, command.metadata().comparator); - if (null != command.index) - IndexMetadata.serializer.serialize(command.index, out, version); + if (null != command.indexQueryPlan) + IndexMetadata.serializer.serialize(command.indexQueryPlan.getFirst().getIndexMetadata(), out, version); command.serializeSelection(out, version); } @@ -993,9 +1038,9 @@ public ReadCommand deserialize(DataInputPlus in, int version) throws IOException // better complain loudly than doing the wrong thing. if (isForThrift(flags)) throw new IllegalStateException("Received a command with the thrift flag set. " - + "This means thrift is in use in a mixed 3.0/3.X and 4.0+ cluster, " - + "which is unsupported. Make sure to stop using thrift before " - + "upgrading to 4.0"); + + "This means thrift is in use in a mixed 3.0/3.X and 4.0+ cluster, " + + "which is unsupported. Make sure to stop using thrift before " + + "upgrading to 4.0"); boolean hasIndex = hasIndex(flags); int digestVersion = isDigest ? (int)in.readUnsignedVInt() : 0; @@ -1004,9 +1049,16 @@ public ReadCommand deserialize(DataInputPlus in, int version) throws IOException ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata); RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata); DataLimits limits = DataLimits.serializer.deserialize(in, version, metadata.comparator); - IndexMetadata index = hasIndex ? deserializeIndexMetadata(in, version, metadata) : null; + Index.QueryPlan indexQueryPlan = null; + if (hasIndex) + { + IndexMetadata index = deserializeIndexMetadata(in, version, metadata); + Index.Group indexGroup = Keyspace.openAndGetStore(metadata).indexManager.getIndexGroup(index); + if (indexGroup != null) + indexQueryPlan = indexGroup.queryPlanFor(rowFilter); + } - return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, index); + return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, indexQueryPlan); } private IndexMetadata deserializeIndexMetadata(DataInputPlus in, int version, TableMetadata metadata) throws IOException diff --git a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java index 2c28ed9d4b8f..293ced084b32 100644 --- a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java +++ b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java @@ -81,8 +81,10 @@ private void validateTransientStatus(Message message) if (command instanceof SinglePartitionReadCommand) token = ((SinglePartitionReadCommand) command).partitionKey().getToken(); - else + else if (command instanceof PartitionRangeReadCommand) token = ((PartitionRangeReadCommand) command).dataRange().keyRange().right.getToken(); + else + return; Replica replica = Keyspace.open(command.metadata().keyspace) .getReplicationStrategy() diff --git a/src/java/org/apache/cassandra/db/ReadExecutionController.java b/src/java/org/apache/cassandra/db/ReadExecutionController.java index 73ddad8022d8..6e917e8e3511 100644 --- a/src/java/org/apache/cassandra/db/ReadExecutionController.java +++ b/src/java/org/apache/cassandra/db/ReadExecutionController.java @@ -93,7 +93,7 @@ public static ReadExecutionController empty() static ReadExecutionController forCommand(ReadCommand command) { ColumnFamilyStore baseCfs = Keyspace.openAndGetStore(command.metadata()); - ColumnFamilyStore indexCfs = maybeGetIndexCfs(baseCfs, command); + ColumnFamilyStore indexCfs = maybeGetIndexCfs(command); long createdAtNanos = baseCfs.metric.topLocalReadQueryTime.isEnabled() ? clock.now() : NO_SAMPLING; @@ -134,10 +134,14 @@ static ReadExecutionController forCommand(ReadCommand command) } } - private static ColumnFamilyStore maybeGetIndexCfs(ColumnFamilyStore baseCfs, ReadCommand command) + private static ColumnFamilyStore maybeGetIndexCfs(ReadCommand command) { - Index index = command.getIndex(baseCfs); - return index == null ? null : index.getBackingTable().orElse(null); + Index.QueryPlan queryPlan = command.indexQueryPlan(); + if (queryPlan == null) + return null; + + // only the index groups with a single member are allowed to have a backing table + return queryPlan.getFirst().getBackingTable().orElse(null); } public TableMetadata metadata() diff --git a/src/java/org/apache/cassandra/db/ReadQuery.java b/src/java/org/apache/cassandra/db/ReadQuery.java index bd20c26d9fd4..41ac15ee85cf 100644 --- a/src/java/org/apache/cassandra/db/ReadQuery.java +++ b/src/java/org/apache/cassandra/db/ReadQuery.java @@ -23,7 +23,7 @@ import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.pager.QueryPager; import org.apache.cassandra.service.pager.PagingState; import org.apache.cassandra.transport.ProtocolVersion; @@ -48,7 +48,7 @@ public ReadExecutionController executionController() return ReadExecutionController.empty(); } - public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException + public PartitionIterator execute(ConsistencyLevel consistency, QueryState queryState, long queryStartNanoTime) throws RequestExecutionException { return EmptyIterators.partition(); } @@ -140,12 +140,12 @@ public ColumnFilter columnFilter() * Executes the query at the provided consistency level. * * @param consistency the consistency level to achieve for the query. - * @param clientState the {@code ClientState} for the query. In practice, this can be null unless + * @param queryState the {@code QueryState} for the query. In practice, this can be null unless * {@code consistency} is a serial consistency. * * @return the result of the query. */ - public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException; + public PartitionIterator execute(ConsistencyLevel consistency, QueryState queryState, long queryStartNanoTime) throws RequestExecutionException; /** * Execute the query for internal queries (that is, it basically executes the query locally). diff --git a/src/java/org/apache/cassandra/db/RepairedDataInfo.java b/src/java/org/apache/cassandra/db/RepairedDataInfo.java index c136f26eff71..0496a07982f8 100644 --- a/src/java/org/apache/cassandra/db/RepairedDataInfo.java +++ b/src/java/org/apache/cassandra/db/RepairedDataInfo.java @@ -289,7 +289,7 @@ private static class RepairedDataPurger extends PurgeFunction super(nowInSec, cfs.gcBefore(nowInSec), oldestUnrepairedTombstone, - cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones(), + cfs.onlyPurgeRepairedTombstones(), cfs.metadata.get().enforceStrictLiveness()); } diff --git a/src/java/org/apache/cassandra/db/SSTableImporter.java b/src/java/org/apache/cassandra/db/SSTableImporter.java index 989ff12297a7..65255f52c2a3 100644 --- a/src/java/org/apache/cassandra/db/SSTableImporter.java +++ b/src/java/org/apache/cassandra/db/SSTableImporter.java @@ -136,7 +136,8 @@ synchronized List importNewSSTables(Options options) Descriptor newDescriptor = cfs.getUniqueDescriptorFor(entry.getKey(), targetDir); maybeMutateMetadata(entry.getKey(), options); movedSSTables.add(new MovedSSTable(newDescriptor, entry.getKey(), entry.getValue())); - SSTableReader sstable = SSTableReader.moveAndOpenSSTable(cfs, entry.getKey(), newDescriptor, entry.getValue(), options.copyData); + SSTableReader sstable = newDescriptor.getFormat().getReaderFactory() + .moveAndOpenSSTable(cfs, entry.getKey(), newDescriptor, entry.getValue(), options.copyData); newSSTablesPerDirectory.add(sstable); } catch (Throwable t) @@ -181,8 +182,15 @@ synchronized List importNewSSTables(Options options) cfs.getTracker().addSSTables(newSSTables); for (SSTableReader reader : newSSTables) { - if (options.invalidateCaches && cfs.isRowCacheEnabled()) - invalidateCachesForSSTable(reader.descriptor); + try + { + if (options.invalidateCaches && cfs.isRowCacheEnabled()) + invalidateCachesForSSTable(reader); + } + catch (IOException ex) + { + throw new RuntimeException(ex); + } } } @@ -208,7 +216,7 @@ private File getTargetDirectory(String srcPath, Descriptor descriptor, Set movedSSTables) * Iterates over all keys in the sstable index and invalidates the row cache */ @VisibleForTesting - void invalidateCachesForSSTable(Descriptor desc) + void invalidateCachesForSSTable(SSTableReader reader) throws IOException { - try (KeyIterator iter = new KeyIterator(desc, cfs.metadata())) + try (KeyIterator iter = KeyIterator.forSSTable(reader)) { while (iter.hasNext()) { @@ -335,7 +343,7 @@ private void verifySSTableForImport(Descriptor descriptor, Set compon SSTableReader reader = null; try { - reader = SSTableReader.open(descriptor, components, cfs.metadata); + reader = descriptor.getFormat().getReaderFactory().open(descriptor, components, cfs.metadata); Verifier.Options verifierOptions = Verifier.options() .extendedVerification(extendedVerify) .checkOwnsTokens(verifyTokens) diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java index dca52c0a9d74..5403c7ee1023 100644 --- a/src/java/org/apache/cassandra/db/SerializationHeader.java +++ b/src/java/org/apache/cassandra/db/SerializationHeader.java @@ -25,7 +25,6 @@ import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.marshal.AbstractType; -import org.apache.cassandra.db.marshal.TypeParser; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.exceptions.UnknownColumnException; @@ -38,6 +37,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.serializers.AbstractTypeSerializer; import org.apache.cassandra.utils.ByteBufferUtil; public class SerializationHeader @@ -398,6 +398,8 @@ public EncodingStats getEncodingStats() public static class Serializer implements IMetadataComponentSerializer { + private final AbstractTypeSerializer typeSerializer = new AbstractTypeSerializer(); + public void serializeForMessaging(SerializationHeader header, ColumnFilter selection, DataOutputPlus out, boolean hasStatic) throws IOException { EncodingStats.serializer.serialize(header.stats, out); @@ -462,10 +464,8 @@ public void serialize(Version version, Component header, DataOutputPlus out) thr { EncodingStats.serializer.serialize(header.stats, out); - writeType(header.keyType, out); - out.writeUnsignedVInt(header.clusteringTypes.size()); - for (AbstractType type : header.clusteringTypes) - writeType(type, out); + typeSerializer.serialize(header.keyType, out); + typeSerializer.serializeList(header.clusteringTypes, out); writeColumnsWithTypes(header.staticColumns, out); writeColumnsWithTypes(header.regularColumns, out); @@ -476,17 +476,11 @@ public Component deserialize(Version version, DataInputPlus in) throws IOExcepti { EncodingStats stats = EncodingStats.serializer.deserialize(in); - AbstractType keyType = readType(in); - int size = (int)in.readUnsignedVInt(); - List> clusteringTypes = new ArrayList<>(size); - for (int i = 0; i < size; i++) - clusteringTypes.add(readType(in)); - - Map> staticColumns = new LinkedHashMap<>(); - Map> regularColumns = new LinkedHashMap<>(); + AbstractType keyType = typeSerializer.deserialize(in); + List> clusteringTypes = typeSerializer.deserializeList(in); - readColumnsWithType(in, staticColumns); - readColumnsWithType(in, regularColumns); + Map> staticColumns = readColumnsWithType(in); + Map> regularColumns = readColumnsWithType(in); return new Component(keyType, clusteringTypes, staticColumns, regularColumns, stats); } @@ -496,10 +490,10 @@ public int serializedSize(Version version, Component header) { int size = EncodingStats.serializer.serializedSize(header.stats); - size += sizeofType(header.keyType); + size += typeSerializer.serializedSize(header.keyType); size += TypeSizes.sizeofUnsignedVInt(header.clusteringTypes.size()); for (AbstractType type : header.clusteringTypes) - size += sizeofType(type); + size += typeSerializer.serializedSize(type); size += sizeofColumnsWithTypes(header.staticColumns); size += sizeofColumnsWithTypes(header.regularColumns); @@ -512,7 +506,7 @@ private void writeColumnsWithTypes(Map> columns, Dat for (Map.Entry> entry : columns.entrySet()) { ByteBufferUtil.writeWithVIntLength(entry.getKey(), out); - writeType(entry.getValue(), out); + typeSerializer.serialize(entry.getValue(), out); } } @@ -522,36 +516,21 @@ private long sizeofColumnsWithTypes(Map> columns) for (Map.Entry> entry : columns.entrySet()) { size += ByteBufferUtil.serializedSizeWithVIntLength(entry.getKey()); - size += sizeofType(entry.getValue()); + size += typeSerializer.serializedSize(entry.getValue()); } return size; } - private void readColumnsWithType(DataInputPlus in, Map> typeMap) throws IOException + private Map> readColumnsWithType(DataInputPlus in) throws IOException { - int length = (int)in.readUnsignedVInt(); + int length = (int) in.readUnsignedVInt(); + Map> typeMap = new LinkedHashMap<>(length); for (int i = 0; i < length; i++) { ByteBuffer name = ByteBufferUtil.readWithVIntLength(in); - typeMap.put(name, readType(in)); + typeMap.put(name, typeSerializer.deserialize(in)); } - } - - private void writeType(AbstractType type, DataOutputPlus out) throws IOException - { - // TODO: we should have a terser serializaion format. Not a big deal though - ByteBufferUtil.writeWithVIntLength(UTF8Type.instance.decompose(type.toString()), out); - } - - private AbstractType readType(DataInputPlus in) throws IOException - { - ByteBuffer raw = ByteBufferUtil.readWithVIntLength(in); - return TypeParser.parse(UTF8Type.instance.compose(raw)); - } - - private int sizeofType(AbstractType type) - { - return ByteBufferUtil.serializedSizeWithVIntLength(UTF8Type.instance.decompose(type.toString())); + return typeMap; } } } diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index 7dba4d88380e..45ace44db1e6 100644 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@ -31,11 +31,14 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.filter.*; import org.apache.cassandra.db.lifecycle.*; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.db.transform.RTBoundValidator; import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.sstable.format.RowIndexEntry; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableReadsListener; import org.apache.cassandra.io.util.DataInputPlus; @@ -43,7 +46,6 @@ import org.apache.cassandra.metrics.TableMetrics; import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.ColumnMetadata; -import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.*; import org.apache.cassandra.tracing.Tracing; @@ -71,9 +73,9 @@ protected SinglePartitionReadCommand(boolean isDigest, DataLimits limits, DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, - IndexMetadata index) + Index.QueryPlan indexQueryPlan) { - super(Kind.SINGLE_PARTITION, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, index); + super(Kind.SINGLE_PARTITION, isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, indexQueryPlan); assert partitionKey.getPartitioner() == metadata.partitioner; this.partitionKey = partitionKey; this.clusteringIndexFilter = clusteringIndexFilter; @@ -89,7 +91,7 @@ protected SinglePartitionReadCommand(boolean isDigest, * @param limits the limits to use for the query. * @param partitionKey the partition key for the partition to query. * @param clusteringIndexFilter the clustering index filter to use for the query. - * @param indexMetadata explicitly specified index to use for the query + * @param indexQueryPlan explicitly specified index to use for the query * * @return a newly created read command. */ @@ -100,7 +102,7 @@ public static SinglePartitionReadCommand create(TableMetadata metadata, DataLimits limits, DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, - IndexMetadata indexMetadata) + Index.QueryPlan indexQueryPlan) { return new SinglePartitionReadCommand(false, 0, @@ -112,7 +114,7 @@ public static SinglePartitionReadCommand create(TableMetadata metadata, limits, partitionKey, clusteringIndexFilter, - indexMetadata); + indexQueryPlan); } /** @@ -143,7 +145,7 @@ public static SinglePartitionReadCommand create(TableMetadata metadata, limits, partitionKey, clusteringIndexFilter, - findIndex(metadata, rowFilter)); + findIndexQueryPlan(metadata, rowFilter)); } /** @@ -288,7 +290,7 @@ public SinglePartitionReadCommand copy() limits(), partitionKey(), clusteringIndexFilter(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -304,7 +306,7 @@ protected SinglePartitionReadCommand copyAsDigestQuery() limits(), partitionKey(), clusteringIndexFilter(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -320,7 +322,7 @@ protected SinglePartitionReadCommand copyAsTransientQuery() limits(), partitionKey(), clusteringIndexFilter(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -336,7 +338,7 @@ public SinglePartitionReadCommand withUpdatedLimit(DataLimits newLimits) newLimits, partitionKey(), clusteringIndexFilter(), - indexMetadata()); + indexQueryPlan()); } @Override @@ -380,19 +382,24 @@ public SinglePartitionReadCommand forPaging(Clustering lastReturned, DataLimi lastReturned == null ? clusteringIndexFilter() : clusteringIndexFilter.forPaging(metadata().comparator, lastReturned, false)); } - public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException + public PartitionIterator execute(ConsistencyLevel consistency, QueryState queryState, long queryStartNanoTime) throws RequestExecutionException { if (clusteringIndexFilter.isEmpty(metadata().comparator)) return EmptyIterators.partition(); - return StorageProxy.read(Group.one(this), consistency, clientState, queryStartNanoTime); + return StorageProxy.read(Group.one(this), consistency, queryState, queryStartNanoTime); } - protected void recordLatency(TableMetrics metric, long latencyNanos) + protected void recordReadLatency(TableMetrics metric, long latencyNanos) { metric.readLatency.addNano(latencyNanos); } + protected void recordReadRequest(TableMetrics metric) + { + metric.readRequests.inc(); + } + @SuppressWarnings("resource") // we close the created iterator through closing the result of this method (and SingletonUnfilteredPartitionIterator ctor cannot fail) protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController) { @@ -441,7 +448,7 @@ private UnfilteredRowIterator getThroughCache(ColumnFamilyStore cfs, ReadExecuti cfs.metric.rowCacheHit.inc(); Tracing.trace("Row cache hit"); UnfilteredRowIterator unfilteredRowIterator = clusteringIndexFilter().getUnfilteredRowIterator(columnFilter(), cachedPartition); - cfs.metric.updateSSTableIterated(0); + cfs.metric.updateSSTableIterated(0, 0); return unfilteredRowIterator; } @@ -566,10 +573,10 @@ public UnfilteredRowIterator queryMemtableAndDisk(ColumnFamilyStore cfs, ReadExe assert executionController != null && executionController.validForReadOn(cfs); Tracing.trace("Executing single-partition query on {}", cfs.name); - return queryMemtableAndDiskInternal(cfs); + return queryMemtableAndDiskInternal(cfs, System.nanoTime()); } - private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs) + private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs, long startTimeNanos) { /* * We have 2 main strategies: @@ -585,7 +592,7 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs * and generate a digest over their merge, which procludes an early return. */ if (clusteringIndexFilter() instanceof ClusteringIndexNamesFilter && !queriesMulticellType() && !isTrackingRepairedStatus()) - return queryMemtableAndSSTablesInTimestampOrder(cfs, (ClusteringIndexNamesFilter)clusteringIndexFilter()); + return queryMemtableAndSSTablesInTimestampOrder(cfs, (ClusteringIndexNamesFilter)clusteringIndexFilter(), startTimeNanos); Tracing.trace("Acquiring sstable references"); ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, partitionKey())); @@ -649,44 +656,43 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs break; } - if (shouldInclude(sstable)) - { - if (!sstable.isRepaired()) - oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime()); + boolean intersects = intersects(sstable); + boolean hasRequiredStatics = hasRequiredStatics(sstable); + boolean hasPartitionLevelDeletions = hasPartitionLevelDeletions(sstable); - // 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator - @SuppressWarnings("resource") - UnfilteredRowIteratorWithLowerBound iter = makeIterator(cfs, sstable, metricsCollector); - inputCollector.addSSTableIterator(sstable, iter); - mostRecentPartitionTombstone = Math.max(mostRecentPartitionTombstone, - iter.partitionLevelDeletion().markedForDeleteAt()); + if (!intersects && !hasRequiredStatics && !hasPartitionLevelDeletions) + { + continue; } - else + + @SuppressWarnings("resource") + UnfilteredRowIterator iter = intersects + ? makeIterator(cfs, sstable, metricsCollector) + : makeIteratorWithSkippedNonStaticContent(cfs, sstable, metricsCollector); + if (!intersects) { nonIntersectingSSTables++; - // sstable contains no tombstone if maxLocalDeletionTime == Integer.MAX_VALUE, so we can safely skip those entirely - if (sstable.mayHaveTombstones()) - { - // 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator - @SuppressWarnings("resource") - UnfilteredRowIteratorWithLowerBound iter = makeIterator(cfs, sstable, metricsCollector); - // if the sstable contains a partition delete, then we must include it regardless of whether it - // shadows any other data seen locally as we can't guarantee that other replicas have seen it + + if (!hasRequiredStatics) { // => has partition level deletions if (!iter.partitionLevelDeletion().isLive()) { - if (!sstable.isRepaired()) - oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime()); - inputCollector.addSSTableIterator(sstable, iter); includedDueToTombstones++; - mostRecentPartitionTombstone = Math.max(mostRecentPartitionTombstone, - iter.partitionLevelDeletion().markedForDeleteAt()); } else { iter.close(); + continue; } } } + + if (!sstable.isRepaired()) + oldestUnrepairedTombstone = Math.min(oldestUnrepairedTombstone, sstable.getMinLocalDeletionTime()); + + inputCollector.addSSTableIterator(sstable, iter); + if (hasPartitionLevelDeletions) + mostRecentPartitionTombstone = Math.max(mostRecentPartitionTombstone, + iter.partitionLevelDeletion().markedForDeleteAt()); } if (Tracing.isTracing()) @@ -698,7 +704,7 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs StorageHook.instance.reportRead(cfs.metadata().id, partitionKey()); - return withSSTablesIterated(inputCollector.finalizeIterators(cfs, nowInSec(), oldestUnrepairedTombstone), cfs.metric, metricsCollector); + return withSSTablesIterated(inputCollector.finalizeIterators(cfs, nowInSec(), oldestUnrepairedTombstone), cfs.metric, metricsCollector, startTimeNanos); } catch (RuntimeException | Error e) { @@ -714,15 +720,20 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs } } - private boolean shouldInclude(SSTableReader sstable) + private boolean intersects(SSTableReader sstable) { + return clusteringIndexFilter().intersects(sstable.metadata().comparator, sstable.getSSTableMetadata().coveredClustering); + } + + private boolean hasRequiredStatics(SSTableReader sstable) { // If some static columns are queried, we should always include the sstable: the clustering values stats of the sstable // don't tell us if the sstable contains static values in particular. - // TODO: we could record if a sstable contains any static value at all. - if (!columnFilter().fetchedColumns().statics.isEmpty()) - return true; + return !columnFilter().fetchedColumns().statics.isEmpty() && sstable.header.hasStatic(); + } - return clusteringIndexFilter().shouldInclude(sstable); + private boolean hasPartitionLevelDeletions(SSTableReader sstable) + { + return sstable.getSSTableMetadata().hasPartitionLevelDeletions; } private UnfilteredRowIteratorWithLowerBound makeIterator(ColumnFamilyStore cfs, @@ -738,6 +749,19 @@ private UnfilteredRowIteratorWithLowerBound makeIterator(ColumnFamilyStore cfs, } + private UnfilteredRowIterator makeIteratorWithSkippedNonStaticContent(ColumnFamilyStore cfs, + SSTableReader sstable, + SSTableReadsListener listener) + { + return StorageHook.instance.makeRowIterator(cfs, + sstable, + partitionKey(), + Slices.NONE, + columnFilter(), + clusteringIndexFilter().isReversed(), + listener); + } + /** * Return a wrapped iterator that when closed will update the sstables iterated and READ sample metrics. * Note that we cannot use the Transformations framework because they greedily get the static row, which @@ -746,7 +770,8 @@ private UnfilteredRowIteratorWithLowerBound makeIterator(ColumnFamilyStore cfs, @SuppressWarnings("resource") private UnfilteredRowIterator withSSTablesIterated(List iterators, TableMetrics metrics, - SSTableReadMetricsCollector metricsCollector) + SSTableReadMetricsCollector metricsCollector, + long startTimeNanos) { @SuppressWarnings("resource") // Closed through the closing of the result of the caller method. UnfilteredRowIterator merged = UnfilteredRowIterators.merge(iterators); @@ -762,7 +787,7 @@ class UpdateSstablesIterated extends Transformation public void onPartitionClose() { int mergedSSTablesIterated = metricsCollector.getMergedSSTables(); - metrics.updateSSTableIterated(mergedSSTablesIterated); + metrics.updateSSTableIterated(mergedSSTablesIterated, System.nanoTime() - startTimeNanos); Tracing.trace("Merged data from memtables and {} sstables", mergedSSTablesIterated); } }; @@ -788,7 +813,7 @@ private boolean queriesMulticellType() * no collection or counters are included). * This method assumes the filter is a {@code ClusteringIndexNamesFilter}. */ - private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFamilyStore cfs, ClusteringIndexNamesFilter filter) + private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFamilyStore cfs, ClusteringIndexNamesFilter filter, long startTimeNanos) { Tracing.trace("Acquiring sstable references"); ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, partitionKey())); @@ -833,72 +858,52 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam if (filter == null) break; - if (!shouldInclude(sstable)) + boolean intersects = intersects(sstable); + boolean hasRequiredStatics = hasRequiredStatics(sstable); + boolean hasPartitionLevelDeletions = hasPartitionLevelDeletions(sstable); + + if (!intersects && !hasRequiredStatics && !hasPartitionLevelDeletions) { // This mean that nothing queried by the filter can be in the sstable. One exception is the top-level partition deletion // however: if it is set, it impacts everything and must be included. Getting that top-level partition deletion costs us // some seek in general however (unless the partition is indexed and is in the key cache), so we first check if the sstable // has any tombstone at all as a shortcut. - if (!sstable.mayHaveTombstones()) - continue; // no tombstone at all, we can skip that sstable - - // We need to get the partition deletion and include it if it's live. In any case though, we're done with that sstable. - try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, - sstable, - partitionKey(), - filter.getSlices(metadata()), - columnFilter(), - filter.isReversed(), - metricsCollector)) - { - if (!iter.partitionLevelDeletion().isLive()) - { - result = add( - UnfilteredRowIterators.noRowsIterator(iter.metadata(), - iter.partitionKey(), - Rows.EMPTY_STATIC_ROW, - iter.partitionLevelDeletion(), - filter.isReversed()), - result, - filter, - sstable.isRepaired() - ); - } - else - { - result = add( - RTBoundValidator.validate(iter, RTBoundValidator.Stage.SSTABLE, false), - result, - filter, - sstable.isRepaired() - ); - } - } - continue; } try (UnfilteredRowIterator iter = StorageHook.instance.makeRowIterator(cfs, sstable, partitionKey(), - filter.getSlices(metadata()), + intersects ? filter.getSlices(metadata()) : Slices.NONE, columnFilter(), filter.isReversed(), metricsCollector)) { - if (iter.isEmpty()) - continue; + if (!hasRequiredStatics && !intersects && !iter.partitionLevelDeletion().isLive()) // => partitionLevelDelections == true + { + result = add(UnfilteredRowIterators.noRowsIterator(iter.metadata(), + iter.partitionKey(), + Rows.EMPTY_STATIC_ROW, + iter.partitionLevelDeletion(), + filter.isReversed()), + result, + filter, + sstable.isRepaired()); + } + else + { + if (!hasRequiredStatics && iter.isEmpty()) + continue; - result = add( - RTBoundValidator.validate(iter, RTBoundValidator.Stage.SSTABLE, false), - result, - filter, - sstable.isRepaired() - ); + result = add(RTBoundValidator.validate(iter, RTBoundValidator.Stage.SSTABLE, false), + result, + filter, + sstable.isRepaired()); + } } } - cfs.metric.updateSSTableIterated(metricsCollector.getMergedSSTables()); + cfs.metric.updateSSTableIterated(metricsCollector.getMergedSSTables(), System.nanoTime() - startTimeNanos); if (result == null || result.isEmpty()) return EmptyIterators.unfilteredRow(metadata(), partitionKey(), false); @@ -1113,9 +1118,9 @@ public static Group one(SinglePartitionReadCommand command) return new Group(Collections.singletonList(command), command.limits()); } - public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException + public PartitionIterator execute(ConsistencyLevel consistency, QueryState queryState, long queryStartNanoTime) throws RequestExecutionException { - return StorageProxy.read(this, consistency, clientState, queryStartNanoTime); + return StorageProxy.read(this, consistency, queryState, queryStartNanoTime); } } @@ -1131,12 +1136,12 @@ public ReadCommand deserialize(DataInputPlus in, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, - IndexMetadata index) + Index.QueryPlan indexQueryPlan) throws IOException { DecoratedKey key = metadata.partitioner.decorateKey(metadata.partitionKeyType.readBuffer(in, DatabaseDescriptor.getMaxValueSize())); ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata); - return new SinglePartitionReadCommand(isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, index); + return new SinglePartitionReadCommand(isDigest, digestVersion, acceptsTransient, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter, indexQueryPlan); } } @@ -1152,7 +1157,7 @@ private static final class SSTableReadMetricsCollector implements SSTableReadsLi private int mergedSSTables; @Override - public void onSSTableSelected(SSTableReader sstable, RowIndexEntry indexEntry, SelectionReason reason) + public void onSSTableSelected(SSTableReader sstable, RowIndexEntry indexEntry, SelectionReason reason) { sstable.incrementReadCount(); mergedSSTables++; diff --git a/src/java/org/apache/cassandra/db/Slice.java b/src/java/org/apache/cassandra/db/Slice.java index 8956bd178e32..be848c961bf8 100644 --- a/src/java/org/apache/cassandra/db/Slice.java +++ b/src/java/org/apache/cassandra/db/Slice.java @@ -22,7 +22,6 @@ import java.util.*; import org.apache.cassandra.db.marshal.AbstractType; -import org.apache.cassandra.db.marshal.ByteArrayAccessor; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; @@ -47,9 +46,9 @@ public boolean includes(ClusteringComparator comparator, ClusteringPrefix clu } @Override - public boolean intersects(ClusteringComparator comparator, List minClusteringValues, List maxClusteringValues) + public boolean intersects(ClusteringComparator comparator, Slice other) { - return true; + return !other.isEmpty(comparator); } @Override @@ -107,6 +106,13 @@ public static Slice make(Clustering start, Clustering end) return new Slice(ClusteringBound.inclusiveStartOf(start), ClusteringBound.inclusiveEndOf(end)); } + public static Slice make(ClusteringPrefix start, ClusteringPrefix end) + { + // This doesn't give us what we want with the clustering prefix + assert start != Clustering.STATIC_CLUSTERING && end != Clustering.STATIC_CLUSTERING; + return make(start.asStartBound(), end.asEndBound()); + } + public ClusteringBound start() { return start; @@ -230,20 +236,25 @@ public Slice forPaging(ClusteringComparator comparator, Clustering lastReturn } /** - * Given the per-clustering column minimum and maximum value a sstable contains, whether or not this slice potentially - * intersects that sstable or not. + * Whether this slice and the provided slice intersects. * * @param comparator the comparator for the table this is a slice of. - * @param minClusteringValues the smallest values for each clustering column that a sstable contains. - * @param maxClusteringValues the biggest values for each clustering column that a sstable contains. + * @param other the other slice to check intersection with. * - * @return whether the slice might intersects with the sstable having {@code minClusteringValues} and - * {@code maxClusteringValues}. + * @return whether this slice intersects {@code other}. */ - public boolean intersects(ClusteringComparator comparator, List minClusteringValues, List maxClusteringValues) + public boolean intersects(ClusteringComparator comparator, Slice other) { - // If this slice starts after max clustering or ends before min clustering, it can't intersect - return start.compareTo(comparator, maxClusteringValues) <= 0 && end.compareTo(comparator, minClusteringValues) >= 0; + // Empty slices never intersect anything (and we have to special case it as there is many ways to build an + // empty slice; for instance, without this, (0, 0) would intersect Slice.ALL or [-1, 1]). + if (isEmpty(comparator) || other.isEmpty(comparator)) + return false; + + // Otherwise, the slice intersects if they contains more than just their boundaries. That is, the comparison + // below needs to be strict, because for instance, a=[0, 3] and b=(3, 5] do not intersects, yet the end of a is + // equal to end start of b as far as `ClusteringPrefix.Kind#compare` goes (see the javadoc on that method for + // why that is). + return comparator.compare(start, other.end) < 0 && comparator.compare(end, other.start) > 0; } public String toString(ClusteringComparator comparator) diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java index 441a5d3d88da..04e4eb13fd46 100644 --- a/src/java/org/apache/cassandra/db/Slices.java +++ b/src/java/org/apache/cassandra/db/Slices.java @@ -94,6 +94,16 @@ public static Slices with(ClusteringComparator comparator, Slice slice) */ public abstract Slice get(int i); + public ClusteringBound start() + { + return get(0).start(); + } + + public ClusteringBound end() + { + return get(size() - 1).end(); + } + /** * Returns slices for continuing the paging of those slices given the last returned clustering prefix. * @@ -128,18 +138,13 @@ public static Slices with(ClusteringComparator comparator, Slice slice) */ public abstract boolean selects(Clustering clustering); - /** - * Given the per-clustering column minimum and maximum value a sstable contains, whether or not this slices potentially - * intersects that sstable or not. + * Checks whether any of the slices intersects witht the given one. * - * @param minClusteringValues the smallest values for each clustering column that a sstable contains. - * @param maxClusteringValues the biggest values for each clustering column that a sstable contains. - * - * @return whether the slices might intersects with the sstable having {@code minClusteringValues} and - * {@code maxClusteringValues}. + * @return {@code true} if there exists a slice which ({@link Slice#intersects(ClusteringComparator, Slice)}) with + * the provided slice */ - public abstract boolean intersects(List minClusteringValues, List maxClusteringValues); + public abstract boolean intersects(Slice slice); public abstract String toCQLString(TableMetadata metadata); @@ -439,11 +444,12 @@ private Slices forReversePaging(ClusteringComparator comparator, Clustering l return Slices.NONE; } - public boolean intersects(List minClusteringValues, List maxClusteringValues) + @Override + public boolean intersects(Slice slice) { - for (Slice slice : this) + for (Slice s : this) { - if (slice.intersects(comparator, minClusteringValues, maxClusteringValues)) + if (s.intersects(comparator, slice)) return true; } return false; @@ -748,7 +754,8 @@ public InOrderTester inOrderTester(boolean reversed) return trivialTester; } - public boolean intersects(List minClusteringValues, List maxClusteringValues) + @Override + public boolean intersects(Slice slice) { return true; } @@ -828,6 +835,12 @@ public boolean intersects(List minClusteringValues, List return false; } + @Override + public boolean intersects(Slice slice) + { + return false; + } + public Iterator iterator() { return Collections.emptyIterator(); diff --git a/src/java/org/apache/cassandra/db/SortedLocalRanges.java b/src/java/org/apache/cassandra/db/SortedLocalRanges.java new file mode 100644 index 000000000000..55c634ce3e49 --- /dev/null +++ b/src/java/org/apache/cassandra/db/SortedLocalRanges.java @@ -0,0 +1,242 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Splitter; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.locator.TokenMetadata; +import org.apache.cassandra.service.PendingRangeCalculatorService; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.FBUtilities; + +/** + * This class contains the local ranges for a given table, sorted. + */ +public class SortedLocalRanges +{ + private static final Logger logger = LoggerFactory.getLogger(SortedLocalRanges.class); + + private final StorageService storageService; + private final ColumnFamilyStore cfs; + private final long ringVersion; + private final List ranges; + private final Map> splits; + + private volatile boolean valid; + + public SortedLocalRanges(StorageService storageService, ColumnFamilyStore cfs, long ringVersion, List ranges) + { + this.storageService = storageService; + this.cfs = cfs; + this.ringVersion = ringVersion; + + List sortedRanges = new ArrayList<>(ranges.size()); + for (Splitter.WeightedRange range : ranges) + { + for (Range unwrapped : range.range().unwrap()) + { + sortedRanges.add(new Splitter.WeightedRange(range.weight(), unwrapped)); + } + } + sortedRanges.sort(Comparator.comparing(Splitter.WeightedRange::left)); + + this.ranges = sortedRanges; + this.splits = new ConcurrentHashMap<>(); + this.valid = true; + } + + /** + * Create a set of sorted local ranges based on the current token metadata and ring version. + * + * This method should preferably only be called by {@link ColumnFamilyStore} because later on, + * ranges may need invalidating, see {@link this#invalidate()} and so a reference must be + * kept to ranges that are passed around, and current cfs does this. + */ + static SortedLocalRanges create(ColumnFamilyStore cfs) + { + StorageService storageService = StorageService.instance; + RangesAtEndpoint localRanges; + List weightedRanges; + long ringVersion; + TokenMetadata tmd; + + do + { + tmd = storageService.getTokenMetadata(); + ringVersion = tmd.getRingVersion(); + localRanges = getLocalRanges(cfs, tmd); + + weightedRanges = new ArrayList<>(localRanges.size()); + for (Range r : localRanges.onlyFull().ranges()) + weightedRanges.add(new Splitter.WeightedRange(1.0, r)); + + for (Range r : localRanges.onlyTransient().ranges()) + weightedRanges.add(new Splitter.WeightedRange(0.1, r)); + + if (logger.isTraceEnabled()) + logger.trace("Got local ranges {} (ringVersion = {})", localRanges, ringVersion); + } + while (ringVersion != tmd.getRingVersion()); // if ringVersion is different here it means that + // it might have changed before we calculated localRanges - recalculate + + return new SortedLocalRanges(storageService, cfs, ringVersion, weightedRanges); + } + + private static RangesAtEndpoint getLocalRanges(ColumnFamilyStore cfs, TokenMetadata tmd) + { + RangesAtEndpoint localRanges; + if (StorageService.instance.isBootstrapMode() + && !StorageService.isReplacingSameAddress()) // When replacing same address, the node marks itself as UN locally + { + PendingRangeCalculatorService.instance.blockUntilFinished(); + localRanges = tmd.getPendingRanges(cfs.keyspace.getName(), FBUtilities.getBroadcastAddressAndPort()); + } + else + { + // Reason we use use the future settled TMD is that if we decommission a node, we want to stream + // from that node to the correct location on disk, if we didn't, we would put new files in the wrong places. + // We do this to minimize the amount of data we need to move in rebalancedisks once everything settled + localRanges = cfs.keyspace.getReplicationStrategy().getAddressReplicas(tmd.cloneAfterAllSettled(), FBUtilities.getBroadcastAddressAndPort()); + } + return localRanges; + } + + @VisibleForTesting + public static SortedLocalRanges forTesting(ColumnFamilyStore cfs, List ranges) + { + return new SortedLocalRanges(null, cfs, 0, ranges); + } + + /** + * check if the given disk boundaries are out of date due not being set or to having too old diskVersion/ringVersion + */ + public boolean isOutOfDate() + { + return !valid || ringVersion != storageService.getTokenMetadata().getRingVersion(); + } + + public void invalidate() + { + this.valid = false; + } + + public List getRanges() + { + return ranges; + } + + public long getRingVersion() + { + return ringVersion; + } + + /** + * Split the local ranges into the given number of parts. + * + * @param numParts the number of parts to split into + * + * @return a list of positions into which the local ranges were split + */ + public List split(int numParts) + { + return splits.computeIfAbsent(numParts, this::doSplit); + } + + private List doSplit(int numParts) + { + Splitter splitter = cfs.getPartitioner().splitter().orElse(null); + + List boundaries; + if (splitter == null) + { + logger.debug("Could not split local ranges into {} parts for {}.{} (no splitter)", numParts, cfs.getKeyspaceName(), cfs.getTableName()); + boundaries = ranges.stream().map(Splitter.WeightedRange::right).collect(Collectors.toList()); + } + else + { + logger.debug("Splitting local ranges into {} parts for {}.{}", numParts, cfs.getKeyspaceName(), cfs.getTableName()); + boundaries = splitter.splitOwnedRanges(numParts, ranges, Splitter.SplitType.ALWAYS_SPLIT).boundaries; + } + + logger.debug("Boundaries for {}.{}: {} ({} splits)", cfs.getKeyspaceName(), cfs.getTableName(), boundaries, boundaries.size()); + return boundaries.stream().map(Token::maxKeyBound).collect(Collectors.toList()); + } + + /** + * Returns the intersection of this list with the given range. + */ + public List subrange(Range range) + { + return ranges.stream() + .map(r -> { + Range subRange = r.range().intersectionNonWrapping(range); + return subRange == null ? null : new Splitter.WeightedRange(r.weight(), subRange); + }) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + SortedLocalRanges that = (SortedLocalRanges) o; + if (ringVersion != that.ringVersion) + return false; + + if (!cfs.equals(that.cfs)) + return false; + + return ranges.equals(that.ranges); + } + + public int hashCode() + { + int result = cfs.hashCode(); + result = 31 * result + Long.hashCode(ringVersion); + result = 31 * result + ranges.hashCode(); + return result; + } + + public String toString() + { + return "LocalRanges{" + + "table=" + cfs.getKeyspaceName() + "." + cfs.getTableName() + + ", ring version=" + ringVersion + + ", num ranges=" + ranges.size() + '}'; + } + + public ColumnFamilyStore getCfs() + { + return cfs; + } +} diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java index cbb708465c5e..9f6620ec567b 100644 --- a/src/java/org/apache/cassandra/db/SystemKeyspace.java +++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java @@ -234,7 +234,7 @@ private SystemKeyspace() + "columnfamily_name text," + "compacted_at timestamp," + "keyspace_name text," - + "rows_merged map," + + "rows_merged map," // Note that we currently store partitions, not rows! + "PRIMARY KEY ((id)))") .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7)) .build(); @@ -501,25 +501,27 @@ public static void persistLocalMetadata() DatabaseDescriptor.getStoragePort()); } - public static void updateCompactionHistory(String ksname, + public static void updateCompactionHistory(UUID id, + String ksname, String cfname, long compactedAt, long bytesIn, long bytesOut, - Map rowsMerged) + Map partitionsMerged) { // don't write anything when the history table itself is compacted, since that would in turn cause new compactions if (ksname.equals("system") && cfname.equals(COMPACTION_HISTORY)) return; + // For historical reasons (pre 3.0 refactor) we call the final field rows_merged but we actually store partitions! String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, compacted_at, bytes_in, bytes_out, rows_merged) VALUES (?, ?, ?, ?, ?, ?, ?)"; executeInternal(format(req, COMPACTION_HISTORY), - UUIDGen.getTimeUUID(), + id, ksname, cfname, ByteBufferUtil.bytes(compactedAt), bytesIn, bytesOut, - rowsMerged); + partitionsMerged); } public static TabularData getCompactionHistory() throws OpenDataException @@ -830,7 +832,9 @@ public static void forceBlockingFlush(String ...cfnames) for (String cfname : cfnames) { - futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(cfname).forceFlush()); + futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME) + .getColumnFamilyStore(cfname) + .forceFlush(ColumnFamilyStore.FlushReason.INTERNALLY_FORCED)); } FBUtilities.waitOnFutures(futures); } diff --git a/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java b/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java index e0a58baf637d..80ee20c8528b 100644 --- a/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java +++ b/src/java/org/apache/cassandra/db/SystemKeyspaceMigrator40.java @@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.PageSize; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; @@ -55,6 +56,7 @@ public class SystemKeyspaceMigrator40 private static final Logger logger = LoggerFactory.getLogger(SystemKeyspaceMigrator40.class); + private static final PageSize DEFAULT_PAGE_SIZE = PageSize.inRows(1000); private SystemKeyspaceMigrator40() {} @@ -95,7 +97,7 @@ private static void migratePeers() + " values ( ?, ?, ? , ? , ?, ?, ?, ?, ?, ?, ?, ?)", peersName); - UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000); + UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, DEFAULT_PAGE_SIZE); int transferred = 0; logger.info("Migrating rows from legacy {} to {}", legacyPeersName, peersName); for (UntypedResultSet.Row row : rows) @@ -138,7 +140,7 @@ private static void migratePeerEvents() + " values ( ?, ?, ? )", peerEventsName); - UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000); + UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, DEFAULT_PAGE_SIZE); int transferred = 0; for (UntypedResultSet.Row row : rows) { @@ -173,7 +175,7 @@ static void migrateTransferredRanges() + " values ( ?, ?, ? , ?, ?)", transferredRangesName); - UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000); + UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, DEFAULT_PAGE_SIZE); int transferred = 0; for (UntypedResultSet.Row row : rows) { @@ -209,7 +211,7 @@ static void migrateAvailableRanges() + " values ( ?, ?, ? )", availableRangesName); - UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, 1000); + UntypedResultSet rows = QueryProcessor.executeInternalWithPaging(query, DEFAULT_PAGE_SIZE); int transferred = 0; for (UntypedResultSet.Row row : rows) { diff --git a/src/java/org/apache/cassandra/db/TableWriteHandler.java b/src/java/org/apache/cassandra/db/TableWriteHandler.java index 4e4722192528..e4caebfaebb0 100644 --- a/src/java/org/apache/cassandra/db/TableWriteHandler.java +++ b/src/java/org/apache/cassandra/db/TableWriteHandler.java @@ -23,5 +23,5 @@ public interface TableWriteHandler { - void write(PartitionUpdate update, WriteContext context, UpdateTransaction updateTransaction); + void write(PartitionUpdate update, WriteContext context, boolean updateIndexes); } diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java index 843054195c67..c3bdb900a7e9 100644 --- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java +++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.cassandra.db.marshal.ByteArrayAccessor; +import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.io.util.DataInputPlus; @@ -34,7 +35,7 @@ public class UnfilteredDeserializer { protected final TableMetadata metadata; - protected final DataInputPlus in; + protected final FileDataInput in; protected final DeserializationHelper helper; private final ClusteringPrefix.Deserializer clusteringDeserializer; @@ -44,11 +45,12 @@ public class UnfilteredDeserializer private int nextExtendedFlags; private boolean isReady; private boolean isDone; + private long preparePos; private final Row.Builder builder; private UnfilteredDeserializer(TableMetadata metadata, - DataInputPlus in, + FileDataInput in, SerializationHeader header, DeserializationHelper helper) { @@ -58,10 +60,11 @@ private UnfilteredDeserializer(TableMetadata metadata, this.header = header; this.clusteringDeserializer = new ClusteringPrefix.Deserializer(metadata.comparator, in, header); this.builder = BTreeRow.sortedBuilder(); + this.preparePos = -1; } public static UnfilteredDeserializer create(TableMetadata metadata, - DataInputPlus in, + FileDataInput in, SerializationHeader header, DeserializationHelper helper) { @@ -85,6 +88,7 @@ private void prepareNext() throws IOException if (isDone) return; + preparePos = in.getFilePointer(); nextFlags = in.readUnsignedByte(); if (UnfilteredSerializer.isEndOfPartition(nextFlags)) { @@ -170,4 +174,5 @@ public void skipNext() throws IOException UnfilteredSerializer.serializer.skipRowBody(in); } } + } diff --git a/src/java/org/apache/cassandra/db/VirtualTableReadQuery.java b/src/java/org/apache/cassandra/db/VirtualTableReadQuery.java index ad22a587a53e..7154c4b811df 100644 --- a/src/java/org/apache/cassandra/db/VirtualTableReadQuery.java +++ b/src/java/org/apache/cassandra/db/VirtualTableReadQuery.java @@ -24,7 +24,7 @@ import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; /** * Base class for the {@code ReadQuery} implementations use to query virtual tables. @@ -48,7 +48,7 @@ public ReadExecutionController executionController() @Override public PartitionIterator execute(ConsistencyLevel consistency, - ClientState clientState, + QueryState queryState, long queryStartNanoTime) throws RequestExecutionException { return executeInternal(executionController()); diff --git a/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java b/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java index ba9441ae7f3e..1369c782abc9 100644 --- a/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java +++ b/src/java/org/apache/cassandra/db/VirtualTableSinglePartitionReadQuery.java @@ -34,7 +34,7 @@ import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; /** * A read query that selects a (part of a) single partition of a virtual table. @@ -181,13 +181,13 @@ public static Group one(VirtualTableSinglePartitionReadQuery query) return new Group(Collections.singletonList(query), query.limits()); } - public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException + public PartitionIterator execute(ConsistencyLevel consistency, QueryState queryState, long queryStartNanoTime) throws RequestExecutionException { if (queries.size() == 1) - return queries.get(0).execute(consistency, clientState, queryStartNanoTime); + return queries.get(0).execute(consistency, queryState, queryStartNanoTime); return PartitionIterators.concat(queries.stream() - .map(q -> q.execute(consistency, clientState, queryStartNanoTime)) + .map(q -> q.execute(consistency, queryState, queryStartNanoTime)) .collect(Collectors.toList())); } } diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java index 848c6e021851..803e880ea5e9 100755 --- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java +++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java @@ -403,9 +403,20 @@ private Future flushDataFrom(List segments, boolean force) else if (!flushes.containsKey(dirtyTableId)) { final ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(dirtyTableId); - // can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush, - // no deadlock possibility since switchLock removal - flushes.put(dirtyTableId, force ? cfs.forceFlush() : cfs.forceFlush(maxCommitLogPosition)); + + if (cfs.memtableWritesAreDurable()) + { + // The memtable does not need this data to be preserved (we only wrote it for PITR and CDC) + segment.markClean(dirtyTableId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition()); + } + else + { + // can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush, + // no deadlock possibility since switchLock removal + flushes.put(dirtyTableId, force + ? cfs.forceFlush(ColumnFamilyStore.FlushReason.INTERNALLY_FORCED) + : cfs.forceFlush(maxCommitLogPosition)); + } } } } diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java index a65ef00a11bd..326936cdb5d1 100644 --- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java +++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java @@ -45,7 +45,7 @@ public abstract class AbstractCommitLogService private volatile boolean shutdown = false; // all Allocations written before this time will be synced - protected volatile long lastSyncedAt = System.currentTimeMillis(); + protected volatile long lastSyncedAt; // counts of total written, and pending, log messages private final AtomicLong written = new AtomicLong(0); @@ -68,6 +68,11 @@ public abstract class AbstractCommitLogService */ final long markerIntervalNanos; + /** + * Provides time related functions for commit log syncing scheduling. + */ + protected final MonotonicClock clock; + /** * A flag that callers outside of the sync thread can use to signal they want the commitlog segments * to be flushed to disk. Note: this flag is primarily to support commit log's batch mode, which requires @@ -83,9 +88,9 @@ public abstract class AbstractCommitLogService * * Subclasses may be notified when a sync finishes by using the syncComplete WaitQueue. */ - AbstractCommitLogService(final CommitLog commitLog, final String name, long syncIntervalMillis) + AbstractCommitLogService(final CommitLog commitLog, final String name, long syncIntervalMillis, MonotonicClock clock) { - this (commitLog, name, syncIntervalMillis, false); + this (commitLog, name, syncIntervalMillis, clock, false); } /** @@ -96,10 +101,12 @@ public abstract class AbstractCommitLogService * * @param markHeadersFaster true if the chained markers should be updated more frequently than on the disk sync bounds. */ - AbstractCommitLogService(final CommitLog commitLog, final String name, long syncIntervalMillis, boolean markHeadersFaster) + AbstractCommitLogService(final CommitLog commitLog, final String name, long syncIntervalMillis, MonotonicClock clock, boolean markHeadersFaster) { this.commitLog = commitLog; this.name = name; + this.clock = clock; + this.lastSyncedAt = clock.now(); final long markerIntervalMillis; if (markHeadersFaster && syncIntervalMillis > DEFAULT_MARKER_INTERVAL_MILLIS) @@ -132,7 +139,7 @@ void start() throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %fms", syncIntervalNanos * 1e-6)); shutdown = false; - thread = NamedThreadFactory.createThread(new SyncRunnable(MonotonicClock.preciseTime), name); + thread = NamedThreadFactory.createThread(new SyncRunnable(clock), name); thread.start(); } @@ -168,7 +175,7 @@ boolean sync() { // sync and signal long pollStarted = clock.now(); - boolean flushToDisk = lastSyncedAt + syncIntervalNanos <= pollStarted || shutdownRequested || syncRequested; + boolean flushToDisk = lastSyncedAt + syncIntervalNanos - pollStarted <= 0 || shutdownRequested || syncRequested; if (flushToDisk) { // in this branch, we want to flush the commit log to disk @@ -192,7 +199,7 @@ boolean sync() return false; long wakeUpAt = pollStarted + markerIntervalNanos; - if (wakeUpAt > now) + if (wakeUpAt - now > 0) LockSupport.parkNanos(wakeUpAt - now); } catch (Throwable t) @@ -218,7 +225,7 @@ boolean maybeLogFlushLag(long pollStarted, long now) // this is the timestamp by which we should have completed the flush long maxFlushTimestamp = pollStarted + syncIntervalNanos; - if (maxFlushTimestamp > now) + if (maxFlushTimestamp - now > 0) return false; // if we have lagged noticeably, update our lag counter @@ -229,7 +236,7 @@ boolean maybeLogFlushLag(long pollStarted, long now) syncCount = 1; totalSyncDuration = flushDuration; } - syncExceededIntervalBy += now - maxFlushTimestamp; + syncExceededIntervalBy += Math.abs(now - maxFlushTimestamp); lagCount++; if (firstLagAt > 0) @@ -241,7 +248,7 @@ boolean maybeLogFlushLag(long pollStarted, long now) 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", Math.abs(now - firstLagAt) * 1e-9d), String.format("%.2f", totalSyncDuration * 1e-6d / syncCount), lagCount, String.format("%.2f", syncExceededIntervalBy * 1e-6d / lagCount)); @@ -292,7 +299,7 @@ public void shutdown() */ public void syncBlocking() { - long requestTime = System.nanoTime(); + long requestTime = clock.now(); requestExtraSync(); awaitSyncAt(requestTime, null); } @@ -302,12 +309,12 @@ void awaitSyncAt(long syncTime, Context context) do { WaitQueue.Signal signal = context != null ? syncComplete.register(context) : syncComplete.register(); - if (lastSyncedAt < syncTime) + if (lastSyncedAt - syncTime < 0) signal.awaitUninterruptibly(); else signal.cancel(); } - while (lastSyncedAt < syncTime); + while (lastSyncedAt - syncTime < 0); } 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 78bf30c336c6..e354b925a036 100644 --- a/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java +++ b/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java @@ -17,6 +17,8 @@ */ package org.apache.cassandra.db.commitlog; +import org.apache.cassandra.utils.MonotonicClock; + class BatchCommitLogService extends AbstractCommitLogService { /** @@ -26,9 +28,9 @@ class BatchCommitLogService extends AbstractCommitLogService */ private static final int POLL_TIME_MILLIS = 1000; - public BatchCommitLogService(CommitLog commitLog) + public BatchCommitLogService(CommitLog commitLog, MonotonicClock clock) { - super(commitLog, "COMMIT-LOG-WRITER", POLL_TIME_MILLIS); + super(commitLog, "COMMIT-LOG-WRITER", POLL_TIME_MILLIS, clock); } protected void maybeWaitForSync(CommitLogSegment.Allocation alloc) diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java index 7670c5e1a1f5..3fdfaf44c127 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java @@ -46,6 +46,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.MonotonicClock; import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation; import static org.apache.cassandra.db.commitlog.CommitLogSegment.CommitLogSegmentFileComparator; @@ -72,6 +73,9 @@ public class CommitLog implements CommitLogMBean volatile Configuration configuration; private boolean started = false; + @VisibleForTesting + final MonotonicClock clock; + private static CommitLog construct() { CommitLog log = new CommitLog(CommitLogArchiver.construct(), DatabaseDescriptor.getCommitLogSegmentMgrProvider()); @@ -96,16 +100,18 @@ private static CommitLog construct() this.archiver = archiver; metrics = new CommitLogMetrics(); + this.clock = MonotonicClock.preciseTime; + switch (DatabaseDescriptor.getCommitLogSync()) { case periodic: - executor = new PeriodicCommitLogService(this); + executor = new PeriodicCommitLogService(this, clock); break; case batch: - executor = new BatchCommitLogService(this); + executor = new BatchCommitLogService(this, clock); break; case group: - executor = new GroupCommitLogService(this); + executor = new GroupCommitLogService(this, clock); break; default: throw new IllegalArgumentException("Unknown commitlog service type: " + DatabaseDescriptor.getCommitLogSync()); @@ -194,10 +200,10 @@ public int recoverFiles(File... clogs) throws IOException return replayer.blockForWrites(); } - public void recoverPath(String path) throws IOException + public void recoverPath(String path, boolean tolerateTruncation) throws IOException { CommitLogReplayer replayer = CommitLogReplayer.construct(this, getLocalHostId()); - replayer.replayPath(new File(path), false); + replayer.replayPath(new File(path), tolerateTruncation); replayer.blockForWrites(); } @@ -211,7 +217,7 @@ private static UUID getLocalHostId() */ public void recover(String path) throws IOException { - recoverPath(path); + recoverPath(path, false); } /** diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java index 700f12a242ca..159cd2d696fe 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java @@ -60,13 +60,17 @@ public class CommitLogDescriptor // We don't support anything pre-3.0 public static final int VERSION_30 = 6; public static final int VERSION_40 = 7; + // For compatibility with CNDB + public static final int VERSION_DSE_68 = 680; + // Stargazer 1.0 messaging + public static final int VERSION_SG_10 = 100; /** * Increment this number if there is a changes in the commit log disc layout or MessagingVersion changes. * Note: make sure to handle {@link #getMessagingVersion()} */ @VisibleForTesting - public static final int current_version = VERSION_40; + public static final int current_version = VERSION_SG_10; final int version; public final long id; @@ -208,7 +212,10 @@ public int getMessagingVersion() case VERSION_30: return MessagingService.VERSION_30; case VERSION_40: + case VERSION_DSE_68: return MessagingService.VERSION_40; + case VERSION_SG_10: + return MessagingService.VERSION_SG_10; default: throw new IllegalStateException("Unknown commitlog version " + version); } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogPosition.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogPosition.java index 3ffb04ceae57..39f1005a0c6d 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogPosition.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogPosition.java @@ -95,12 +95,6 @@ public String toString() ')'; } - public CommitLogPosition clone() - { - return new CommitLogPosition(segmentId, position); - } - - public static class CommitLogPositionSerializer implements ISerializer { public void serialize(CommitLogPosition clsp, DataOutputPlus out) throws IOException diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java index 39777ec46ca1..e83217cbf119 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java @@ -130,7 +130,16 @@ public static CommitLogReplayer construct(CommitLog commitLog, UUID localHostId) } } - IntervalSet filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt, localHostId); + IntervalSet filter; + if (!cfs.memtableWritesAreDurable()) + { + filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt, localHostId); + } + else + { + // everything is persisted and restored by the memtable itself + filter = new IntervalSet<>(CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); + } cfPersisted.put(cfs.metadata.id, filter); } CommitLogPosition globalPosition = firstNotCovered(cfPersisted.values()); @@ -212,12 +221,14 @@ public int blockForWrites() if (keyspace.getName().equals(SchemaConstants.SYSTEM_KEYSPACE_NAME)) flushingSystem = true; - futures.addAll(keyspace.flush()); + futures.addAll(keyspace.flush(ColumnFamilyStore.FlushReason.STARTUP)); } // also flush batchlog incase of any MV updates if (!flushingSystem) - futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlush()); + futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME) + .getColumnFamilyStore(SystemKeyspace.BATCHES) + .forceFlush(ColumnFamilyStore.FlushReason.INTERNALLY_FORCED)); FBUtilities.waitOnFutures(futures); diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java index e23a915ba355..de4f135583ac 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java @@ -87,9 +87,9 @@ protected SyncSegment computeNext() { while (true) { + final int currentStart = end; try { - final int currentStart = end; end = readSyncMarker(descriptor, currentStart, reader); if (end == -1) { @@ -133,6 +133,13 @@ protected SyncSegment computeNext() throw new RuntimeException(ioe); } } + + // if we've not been able to read the sync marker, or the file is truncated, + // then return end of data, otherwise continue the loop + if (currentStart == end) + { + return endOfData(); + } } } } diff --git a/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java index a76923e581e0..056bc6c88cbe 100644 --- a/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java +++ b/src/java/org/apache/cassandra/db/commitlog/GroupCommitLogService.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.commitlog; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.utils.MonotonicClock; /** * A commitlog service that will block returning an ACK back to the a coordinator/client @@ -26,9 +27,9 @@ */ public class GroupCommitLogService extends AbstractCommitLogService { - public GroupCommitLogService(CommitLog commitLog) + public GroupCommitLogService(CommitLog commitLog, MonotonicClock clock) { - super(commitLog, "GROUP-COMMIT-LOG-WRITER", (int) DatabaseDescriptor.getCommitLogSyncGroupWindow()); + super(commitLog, "GROUP-COMMIT-LOG-WRITER", (int) DatabaseDescriptor.getCommitLogSyncGroupWindow(), clock); } protected void maybeWaitForSync(CommitLogSegment.Allocation alloc) diff --git a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java index e94c616e444f..c33624cde41d 100644 --- a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java +++ b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java @@ -20,21 +20,22 @@ import java.util.concurrent.TimeUnit; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.utils.MonotonicClock; class PeriodicCommitLogService extends AbstractCommitLogService { private static final long blockWhenSyncLagsNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getPeriodicCommitLogSyncBlock()); - public PeriodicCommitLogService(final CommitLog commitLog) + public PeriodicCommitLogService(final CommitLog commitLog, MonotonicClock clock) { - super(commitLog, "PERIODIC-COMMIT-LOG-SYNCER", DatabaseDescriptor.getCommitLogSyncPeriod(), + super(commitLog, "PERIODIC-COMMIT-LOG-SYNCER", DatabaseDescriptor.getCommitLogSyncPeriod(), clock, !(commitLog.configuration.useCompression() || commitLog.configuration.useEncryption())); } protected void maybeWaitForSync(CommitLogSegment.Allocation alloc) { - long expectedSyncTime = System.nanoTime() - blockWhenSyncLagsNanos; - if (lastSyncedAt < expectedSyncTime) + long expectedSyncTime = clock.now() - blockWhenSyncLagsNanos; + if (lastSyncedAt - expectedSyncTime < 0) { pending.incrementAndGet(); awaitSyncAt(expectedSyncTime, commitLog.metrics.waitingOnCommit.time()); diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java index 0b37c225c4ca..80770d702946 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java @@ -17,69 +17,56 @@ */ package org.apache.cassandra.db.compaction; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; -import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.SerializationHeader; -import org.apache.cassandra.index.Index; -import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; -import org.apache.cassandra.io.sstable.Descriptor; -import org.apache.cassandra.io.sstable.SSTableMultiWriter; -import org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter; -import org.apache.cassandra.io.sstable.format.SSTableReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.Tracker; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.ScannerList; +import org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; -import org.apache.cassandra.io.sstable.metadata.StatsMetadata; -import org.apache.cassandra.schema.CompactionParams; -/** - * Pluggable compaction strategy determines how SSTables get merged. - * - * There are two main goals: - * - perform background compaction constantly as needed; this typically makes a tradeoff between - * i/o done by compaction, and merging done at read time. - * - perform a full (maximum possible) compaction if requested by the user - */ -public abstract class AbstractCompactionStrategy -{ - private static final Logger logger = LoggerFactory.getLogger(AbstractCompactionStrategy.class); - - protected static final float DEFAULT_TOMBSTONE_THRESHOLD = 0.2f; - // minimum interval needed to perform tombstone removal compaction in seconds, default 86400 or 1 day. - protected static final long DEFAULT_TOMBSTONE_COMPACTION_INTERVAL = 86400; - protected static final boolean DEFAULT_UNCHECKED_TOMBSTONE_COMPACTION_OPTION = false; - protected static final boolean DEFAULT_LOG_ALL_OPTION = false; +import static org.apache.cassandra.db.ColumnFamilyStore.nonSuspectAndNotInPredicate; - protected static final String TOMBSTONE_THRESHOLD_OPTION = "tombstone_threshold"; - protected static final String TOMBSTONE_COMPACTION_INTERVAL_OPTION = "tombstone_compaction_interval"; - // disable range overlap check when deciding if an SSTable is candidate for tombstone compaction (CASSANDRA-6563) - protected static final String UNCHECKED_TOMBSTONE_COMPACTION_OPTION = "unchecked_tombstone_compaction"; - protected static final String LOG_ALL_OPTION = "log_all"; - protected static final String COMPACTION_ENABLED = "enabled"; - public static final String ONLY_PURGE_REPAIRED_TOMBSTONES = "only_purge_repaired_tombstones"; +abstract class AbstractCompactionStrategy implements CompactionStrategy +{ + public static final Class CONTAINER_CLASS = CompactionStrategyManager.class; - protected Map options; + protected static final Logger logger = LoggerFactory.getLogger(AbstractCompactionStrategy.class); + protected final CompactionStrategyOptions options; protected final ColumnFamilyStore cfs; - protected float tombstoneThreshold; - protected long tombstoneCompactionInterval; - protected boolean uncheckedTombstoneCompaction; - protected boolean disableTombstoneCompactions = false; - protected boolean logAll = true; + protected final Tracker dataTracker; - private final Directories directories; + protected final CompactionLogger compactionLogger; + protected final Directories directories; + /** + * This class groups all the compaction tasks that are pending, submitted, in progress and completed. + */ + protected final BackgroundCompactions backgroundCompactions; /** * pause/resume/getNextBackgroundTask must synchronize. This guarantees that after pause completes, @@ -91,48 +78,54 @@ public abstract class AbstractCompactionStrategy * * See CASSANDRA-3430 */ - protected boolean isActive = false; + protected volatile boolean isActive = false; - protected AbstractCompactionStrategy(ColumnFamilyStore cfs, Map options) + protected AbstractCompactionStrategy(CompactionStrategyFactory factory, BackgroundCompactions backgroundCompactions, Map options) { - assert cfs != null; - this.cfs = cfs; - this.options = ImmutableMap.copyOf(options); + assert factory != null; + this.cfs = factory.getCfs(); + this.dataTracker = cfs.getTracker(); + this.compactionLogger = factory.getCompactionLogger(); + this.options = new CompactionStrategyOptions(getClass(), options, false); + this.directories = cfs.getDirectories(); + this.backgroundCompactions = backgroundCompactions; + } - /* checks must be repeated here, as user supplied strategies might not call validateOptions directly */ + CompactionStrategyOptions getOptions() + { + return options; + } - try - { - validateOptions(options); - String optionValue = options.get(TOMBSTONE_THRESHOLD_OPTION); - tombstoneThreshold = optionValue == null ? DEFAULT_TOMBSTONE_THRESHOLD : Float.parseFloat(optionValue); - optionValue = options.get(TOMBSTONE_COMPACTION_INTERVAL_OPTION); - tombstoneCompactionInterval = optionValue == null ? DEFAULT_TOMBSTONE_COMPACTION_INTERVAL : Long.parseLong(optionValue); - optionValue = options.get(UNCHECKED_TOMBSTONE_COMPACTION_OPTION); - uncheckedTombstoneCompaction = optionValue == null ? DEFAULT_UNCHECKED_TOMBSTONE_COMPACTION_OPTION : Boolean.parseBoolean(optionValue); - optionValue = options.get(LOG_ALL_OPTION); - logAll = optionValue == null ? DEFAULT_LOG_ALL_OPTION : Boolean.parseBoolean(optionValue); - } - catch (ConfigurationException e) - { - logger.warn("Error setting compaction strategy options ({}), defaults will be used", e.getMessage()); - tombstoneThreshold = DEFAULT_TOMBSTONE_THRESHOLD; - tombstoneCompactionInterval = DEFAULT_TOMBSTONE_COMPACTION_INTERVAL; - uncheckedTombstoneCompaction = DEFAULT_UNCHECKED_TOMBSTONE_COMPACTION_OPTION; - } + public CompactionLogger getCompactionLogger() + { + return compactionLogger; + } - directories = cfs.getDirectories(); + // + // Compaction Observer + // + + @Override + public void onInProgress(CompactionProgress progress) + { + backgroundCompactions.onInProgress(progress); } - public Directories getDirectories() + @Override + public void onCompleted(UUID id) { - return directories; + backgroundCompactions.onCompleted(this, id); } + // + // CompactionStrategy + // + /** * For internal, temporary suspension of background compactions so that we can do exceptional * things like truncate or major compaction */ + @Override public synchronized void pause() { isActive = false; @@ -142,6 +135,7 @@ public synchronized void pause() * For internal, temporary suspension of background compactions so that we can do exceptional * things like truncate or major compaction */ + @Override public synchronized void resume() { isActive = true; @@ -150,6 +144,7 @@ public synchronized void resume() /** * Performs any extra initialization required */ + @Override public void startup() { isActive = true; @@ -158,20 +153,12 @@ public void startup() /** * Releases any resources if this strategy is shutdown (when the CFS is reloaded after a schema change). */ + @Override public void shutdown() { isActive = false; } - /** - * @param gcBefore throw away tombstones older than this - * - * @return the next background/minor compaction task to run; null if nothing to do. - * - * Is responsible for marking its sstables as compaction-pending. - */ - public abstract AbstractCompactionTask getNextBackgroundTask(final int gcBefore); - /** * @param gcBefore throw away tombstones older than this * @@ -180,7 +167,18 @@ public void shutdown() * * Is responsible for marking its sstables as compaction-pending. */ - public abstract Collection getMaximalTask(final int gcBefore, boolean splitOutput); + @Override + @SuppressWarnings("resource") + public synchronized CompactionTasks getMaximalTasks(int gcBefore, boolean splitOutput) + { + Iterable filteredSSTables = Iterables.filter(getSSTables(), sstable -> !sstable.isMarkedSuspect()); + if (Iterables.isEmpty(filteredSSTables)) + return CompactionTasks.empty(); + LifecycleTransaction txn = dataTracker.tryModify(filteredSSTables, OperationType.COMPACTION); + if (txn == null) + return CompactionTasks.empty(); + return CompactionTasks.create(Collections.singleton(createCompactionTask(gcBefore, txn, true, splitOutput))); + } /** * @param sstables SSTables to compact. Must be marked as compacting. @@ -191,317 +189,134 @@ public void shutdown() * * Is responsible for marking its sstables as compaction-pending. */ - public abstract AbstractCompactionTask getUserDefinedTask(Collection sstables, final int gcBefore); - - public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, final int gcBefore, long maxSSTableBytes) + @Override + @SuppressWarnings("resource") + public synchronized CompactionTasks getUserDefinedTasks(Collection sstables, int gcBefore) { - return new CompactionTask(cfs, txn, gcBefore); - } - - /** - * @return the number of background tasks estimated to still be needed for this columnfamilystore - */ - public abstract int getEstimatedRemainingTasks(); - - /** - * @return size in bytes of the largest sstables for this strategy - */ - public abstract long getMaxSSTableBytes(); + assert !sstables.isEmpty(); // checked for by CM.submitUserDefined - /** - * Filters SSTables that are to be excluded from the given collection - * - * @param originalCandidates The collection to check for excluded SSTables - * @return list of the SSTables with excluded ones filtered out - */ - public static List filterSuspectSSTables(Iterable originalCandidates) - { - List filtered = new ArrayList<>(); - for (SSTableReader sstable : originalCandidates) + LifecycleTransaction modifier = dataTracker.tryModify(sstables, OperationType.COMPACTION); + if (modifier == null) { - if (!sstable.isMarkedSuspect()) - filtered.add(sstable); + logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables); + return CompactionTasks.empty(); } - return filtered; - } - - public ScannerList getScanners(Collection sstables, Range range) - { - return range == null ? getScanners(sstables, (Collection>)null) : getScanners(sstables, Collections.singleton(range)); + return CompactionTasks.create(ImmutableList.of(createCompactionTask(gcBefore, modifier, false, false).setUserDefined(true))); } + /** - * Returns a list of KeyScanners given sstables and a range on which to scan. - * The default implementation simply grab one SSTableScanner per-sstable, but overriding this method - * allow for a more memory efficient solution if we know the sstable don't overlap (see - * LeveledCompactionStrategy for instance). + * Create a compaction task for a maximal, user defined or background compaction without aggregates (legacy strategies). + * Background compactions for strategies that extend {@link LegacyAbstractCompactionStrategy.WithAggregates} will use + * {@link LegacyAbstractCompactionStrategy.WithAggregates#createCompactionTask(int, LifecycleTransaction, boolean, boolean)} instead. + * + * @param gcBefore tombstone threshold, older tombstones can be discarded + * @param txn the transaction containing the files to be compacted + * @param isMaximal set to true only when it's a maximal compaction + * @param splitOutput false except for maximal compactions and passed in by the user to indicate to SizeTieredCompactionStrategy to split the out, + * ignored otherwise + * + * @return a compaction task, see {@link AbstractCompactionTask} and sub-classes */ - @SuppressWarnings("resource") - public ScannerList getScanners(Collection sstables, Collection> ranges) - { - ArrayList scanners = new ArrayList(); - try - { - for (SSTableReader sstable : sstables) - scanners.add(sstable.getScanner(ranges)); - } - catch (Throwable t) - { - ISSTableScanner.closeAllAndPropagate(scanners, t); - } - return new ScannerList(scanners); - } - - public String getName() + protected AbstractCompactionTask createCompactionTask(final int gcBefore, LifecycleTransaction txn, boolean isMaximal, boolean splitOutput) { - return getClass().getSimpleName(); + return new CompactionTask(cfs, txn, gcBefore, false, this); } /** - * Replaces sstables in the compaction strategy + * Create a compaction task for operations that are not driven by the strategies. + * + * @param txn the transaction containing the files to be compacted + * @param gcBefore tombstone threshold, older tombstones can be discarded + * @param maxSSTableBytes the maximum size in bytes for an output sstables * - * Note that implementations must be able to handle duplicate notifications here (that removed are already gone and - * added have already been added) - * */ - public synchronized void replaceSSTables(Collection removed, Collection added) + * @return a compaction task, see {@link AbstractCompactionTask} and sub-classes + */ + @Override + public AbstractCompactionTask createCompactionTask(LifecycleTransaction txn, final int gcBefore, long maxSSTableBytes) { - for (SSTableReader remove : removed) - removeSSTable(remove); - addSSTables(added); + return new CompactionTask(cfs, txn, gcBefore, false, this); } /** - * Adds sstable, note that implementations must handle duplicate notifications here (added already being in the compaction strategy) - */ - public abstract void addSSTable(SSTableReader added); - - /** - * Adds sstables, note that implementations must handle duplicate notifications here (added already being in the compaction strategy) + * @return a list of the compaction aggregates, e.g. the levels or buckets. Note that legacy strategies that derive from + * {@link LeveledCompactionStrategy.WithSSTableList} will return an empty list. */ - public synchronized void addSSTables(Iterable added) + public Collection getAggregates() { - for (SSTableReader sstable : added) - addSSTable(sstable); + return backgroundCompactions.getAggregates(); } /** - * Removes sstable from the strategy, implementations must be able to handle the sstable having already been removed. + * @return the total number of background compactions, pending or in progress */ - public abstract void removeSSTable(SSTableReader sstable); - - /** - * Removes sstables from the strategy, implementations must be able to handle the sstables having already been removed. - */ - public void removeSSTables(Iterable removed) + @Override + public int getTotalCompactions() { - for (SSTableReader sstable : removed) - removeSSTable(sstable); + return getEstimatedRemainingTasks() + backgroundCompactions.getCompactionsInProgress().size(); } /** - * Returns the sstables managed by this strategy instance - */ - @VisibleForTesting - protected abstract Set getSSTables(); - - /** - * Called when the metadata has changed for an sstable - for example if the level changed - * - * Not called when repair status changes (which is also metadata), because this results in the - * sstable getting removed from the compaction strategy instance. - * - * @param oldMetadata - * @param sstable + * Return the statistics. Only strategies that implement {@link LegacyAbstractCompactionStrategy.WithAggregates} will provide non-empty statistics, + * the legacy strategies will always have empty statistics. + *

+ * @return statistics about this compaction picks. */ - public void metadataChanged(StatsMetadata oldMetadata, SSTableReader sstable) + @Override + public List getStatistics() { + return ImmutableList.of(backgroundCompactions.getStatistics(this)); } - public static class ScannerList implements AutoCloseable + public static Iterable nonSuspectAndNotIn(Iterable sstables, Set compacting) { - public final List scanners; - public ScannerList(List scanners) - { - this.scanners = scanners; - } - - public long getTotalBytesScanned() - { - long bytesScanned = 0L; - for (int i=0, isize=scanners.size(); i toCompact) + @Override + public int getLevelFanoutSize() { - return getScanners(toCompact, (Collection>)null); + return LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE; // this makes no sense but it's the existing behaviour } /** - * Check if given sstable is worth dropping tombstones at gcBefore. - * Check is skipped if tombstone_compaction_interval time does not elapse since sstable creation and returns false. - * - * @param sstable SSTable to check - * @param gcBefore time to drop tombstones - * @return true if given sstable's tombstones are expected to be removed + * Returns a list of KeyScanners given sstables and a range on which to scan. + * The default implementation simply grab one SSTableScanner per-sstable, but overriding this method + * allow for a more memory efficient solution if we know the sstable don't overlap (see + * LeveledCompactionStrategy for instance). */ - protected boolean worthDroppingTombstones(SSTableReader sstable, int gcBefore) + @SuppressWarnings("resource") + @Override + public ScannerList getScanners(Collection sstables, Collection> ranges) { - if (disableTombstoneCompactions || CompactionController.NEVER_PURGE_TOMBSTONES || cfs.getNeverPurgeTombstones()) - return false; - // since we use estimations to calculate, there is a chance that compaction will not drop tombstones actually. - // if that happens we will end up in infinite compaction loop, so first we check enough if enough time has - // elapsed since SSTable created. - if (System.currentTimeMillis() < sstable.getCreationTimeFor(Component.DATA) + tombstoneCompactionInterval * 1000) - return false; - - double droppableRatio = sstable.getEstimatedDroppableTombstoneRatio(gcBefore); - if (droppableRatio <= tombstoneThreshold) - return false; - - //sstable range overlap check is disabled. See CASSANDRA-6563. - if (uncheckedTombstoneCompaction) - return true; - - Collection overlaps = cfs.getOverlappingLiveSSTables(Collections.singleton(sstable)); - if (overlaps.isEmpty()) - { - // there is no overlap, tombstones are safely droppable - return true; - } - else if (CompactionController.getFullyExpiredSSTables(cfs, Collections.singleton(sstable), overlaps, gcBefore).size() > 0) + ArrayList scanners = new ArrayList(); + try { - return true; + for (SSTableReader sstable : sstables) + scanners.add(sstable.getScanner(ranges)); } - else + catch (Throwable t) { - // what percentage of columns do we expect to compact outside of overlap? - if (sstable.getIndexSummarySize() < 2) - { - // we have too few samples to estimate correct percentage - return false; - } - // first, calculate estimated keys that do not overlap - long keys = sstable.estimatedKeys(); - Set> ranges = new HashSet>(overlaps.size()); - for (SSTableReader overlap : overlaps) - ranges.add(new Range<>(overlap.first.getToken(), overlap.last.getToken())); - long remainingKeys = keys - sstable.estimatedKeysForRanges(ranges); - // next, calculate what percentage of columns we have within those keys - long columns = sstable.getEstimatedCellPerPartitionCount().mean() * remainingKeys; - double remainingColumnsRatio = ((double) columns) / (sstable.getEstimatedCellPerPartitionCount().count() * sstable.getEstimatedCellPerPartitionCount().mean()); - - // return if we still expect to have droppable tombstones in rest of columns - return remainingColumnsRatio * droppableRatio > tombstoneThreshold; + ISSTableScanner.closeAllAndPropagate(scanners, t); } + return new ScannerList(scanners); } - public static Map validateOptions(Map options) throws ConfigurationException + @Override + public String getName() { - String threshold = options.get(TOMBSTONE_THRESHOLD_OPTION); - if (threshold != null) - { - try - { - float thresholdValue = Float.parseFloat(threshold); - if (thresholdValue < 0) - { - throw new ConfigurationException(String.format("%s must be greater than 0, but was %f", TOMBSTONE_THRESHOLD_OPTION, thresholdValue)); - } - } - catch (NumberFormatException e) - { - throw new ConfigurationException(String.format("%s is not a parsable int (base10) for %s", threshold, TOMBSTONE_THRESHOLD_OPTION), e); - } - } - - String interval = options.get(TOMBSTONE_COMPACTION_INTERVAL_OPTION); - if (interval != null) - { - try - { - long tombstoneCompactionInterval = Long.parseLong(interval); - if (tombstoneCompactionInterval < 0) - { - throw new ConfigurationException(String.format("%s must be greater than 0, but was %d", TOMBSTONE_COMPACTION_INTERVAL_OPTION, tombstoneCompactionInterval)); - } - } - catch (NumberFormatException e) - { - throw new ConfigurationException(String.format("%s is not a parsable int (base10) for %s", interval, TOMBSTONE_COMPACTION_INTERVAL_OPTION), e); - } - } - - String unchecked = options.get(UNCHECKED_TOMBSTONE_COMPACTION_OPTION); - if (unchecked != null) - { - if (!unchecked.equalsIgnoreCase("true") && !unchecked.equalsIgnoreCase("false")) - throw new ConfigurationException(String.format("'%s' should be either 'true' or 'false', not '%s'", UNCHECKED_TOMBSTONE_COMPACTION_OPTION, unchecked)); - } - - String logAll = options.get(LOG_ALL_OPTION); - if (logAll != null) - { - if (!logAll.equalsIgnoreCase("true") && !logAll.equalsIgnoreCase("false")) - { - throw new ConfigurationException(String.format("'%s' should either be 'true' or 'false', not %s", LOG_ALL_OPTION, logAll)); - } - } - - String compactionEnabled = options.get(COMPACTION_ENABLED); - if (compactionEnabled != null) - { - if (!compactionEnabled.equalsIgnoreCase("true") && !compactionEnabled.equalsIgnoreCase("false")) - { - throw new ConfigurationException(String.format("enabled should either be 'true' or 'false', not %s", compactionEnabled)); - } - } + return getClass().getSimpleName(); + } - Map uncheckedOptions = new HashMap(options); - uncheckedOptions.remove(TOMBSTONE_THRESHOLD_OPTION); - uncheckedOptions.remove(TOMBSTONE_COMPACTION_INTERVAL_OPTION); - uncheckedOptions.remove(UNCHECKED_TOMBSTONE_COMPACTION_OPTION); - uncheckedOptions.remove(LOG_ALL_OPTION); - uncheckedOptions.remove(COMPACTION_ENABLED); - uncheckedOptions.remove(ONLY_PURGE_REPAIRED_TOMBSTONES); - uncheckedOptions.remove(CompactionParams.Option.PROVIDE_OVERLAPPING_TOMBSTONES.toString()); - return uncheckedOptions; + public static Map validateOptions(Map options) throws ConfigurationException + { + return CompactionStrategyOptions.validateOptions(options); } /** @@ -510,11 +325,12 @@ public static Map validateOptions(Map options) t * as a group. If a given compaction strategy creates sstables which * cannot be merged due to some constraint it must override this method. */ + @Override public Collection> groupSSTablesForAntiCompaction(Collection sstablesToGroup) { int groupSize = 2; List sortedSSTablesToGroup = new ArrayList<>(sstablesToGroup); - Collections.sort(sortedSSTablesToGroup, SSTableReader.sstableComparator); + Collections.sort(sortedSSTablesToGroup, SSTableReader.firstKeyComparator); Collection> groupedSSTables = new ArrayList<>(); Collection currGroup = new ArrayList<>(groupSize); @@ -534,11 +350,6 @@ public Collection> groupSSTablesForAntiCompaction(Coll return groupedSSTables; } - public CompactionLogger.Strategy strategyLogger() - { - return CompactionLogger.Strategy.none; - } - public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, @@ -546,12 +357,13 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, boolean isTransient, MetadataCollector meta, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { - return SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, cfs.metadata, meta, header, indexes, lifecycleNewTracker); + return SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, cfs.metadata, meta, header, indexGroups, lifecycleNewTracker); } + @Override public boolean supportsEarlyOpen() { return true; diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java index 989c21c27c50..04b559501373 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java @@ -17,10 +17,13 @@ */ package org.apache.cassandra.db.compaction; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; import java.util.Set; import java.util.UUID; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.cassandra.db.ColumnFamilyStore; @@ -28,15 +31,21 @@ import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.io.FSDiskFullWriteError; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.WrappedRunnable; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import static com.google.common.base.Throwables.propagate; + + public abstract class AbstractCompactionTask extends WrappedRunnable { protected final ColumnFamilyStore cfs; protected LifecycleTransaction transaction; protected boolean isUserDefined; protected OperationType compactionType; + protected TableOperationObserver opObserver; + protected final List compObservers; /** * @param cfs @@ -48,12 +57,22 @@ public AbstractCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transa this.transaction = transaction; this.isUserDefined = false; this.compactionType = OperationType.COMPACTION; - // enforce contract that caller should mark sstables compacting - Set compacting = transaction.tracker.getCompacting(); - for (SSTableReader sstable : transaction.originals()) - assert compacting.contains(sstable) : sstable.getFilename() + " is not correctly marked compacting"; + this.opObserver = TableOperationObserver.NOOP; + this.compObservers = new ArrayList<>(); + + try + { + // enforce contract that caller should mark sstables compacting + Set compacting = transaction.getCompacting(); + for (SSTableReader sstable : transaction.originals()) + assert compacting.contains(sstable) : sstable.getFilename() + " is not correctly marked compacting"; - validateSSTables(transaction.originals()); + validateSSTables(transaction.originals()); + } + catch (Throwable err) + { + propagate(cleanup(err)); + } } /** @@ -91,15 +110,22 @@ private void validateSSTables(Set sstables) } /** - * executes the task and unmarks sstables compacting + * Executes the task after setting a new observer, normally the observer is the + * compaction manager metrics. */ - public int execute(ActiveCompactionsTracker activeCompactions) + public int execute(TableOperationObserver observer) + { + return setOpObserver(observer).execute(); + } + + /** Executes the task */ + public int execute() { try { - return executeInternal(activeCompactions); + return executeInternal(); } - catch(FSDiskFullWriteError e) + catch (FSDiskFullWriteError e) { RuntimeException cause = new RuntimeException("Converted from FSDiskFullWriteError: " + e.getMessage()); cause.setStackTrace(e.getStackTrace()); @@ -107,12 +133,23 @@ public int execute(ActiveCompactionsTracker activeCompactions) } finally { - transaction.close(); + Throwables.maybeFail(cleanup(null)); } } + + private Throwable cleanup(Throwable err) + { + for (CompactionObserver compObserver : compObservers) + err = Throwables.perform(err, () -> compObserver.onCompleted(transaction.opId())); + + return Throwables.perform(err, () -> transaction.close()); + } + public abstract CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set nonExpiredSSTables); - protected abstract int executeInternal(ActiveCompactionsTracker activeCompactions); + protected abstract int executeInternal(); + + // TODO Eventually these three setters should be passed in to the constructor. public AbstractCompactionTask setUserDefined(boolean isUserDefined) { @@ -126,6 +163,26 @@ public AbstractCompactionTask setCompactionType(OperationType compactionType) return this; } + /** + * Override the NO OP observer, this is normally overridden by the compaction metrics. + */ + AbstractCompactionTask setOpObserver(TableOperationObserver opObserver) + { + this.opObserver = opObserver; + return this; + } + + void addObserver(CompactionObserver compObserver) + { + compObservers.add(compObserver); + } + + @VisibleForTesting + LifecycleTransaction transaction() + { + return transaction; + } + public String toString() { return "CompactionTask(" + transaction + ")"; diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java index 95fc7b85b08f..89c212c7a831 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractStrategyHolder.java @@ -31,7 +31,6 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.index.Index; @@ -50,23 +49,23 @@ */ public abstract class AbstractStrategyHolder { - public static class TaskSupplier implements Comparable + public static class TasksSupplier implements Comparable { private final int numRemaining; - private final Supplier supplier; + private final Supplier> supplier; - TaskSupplier(int numRemaining, Supplier supplier) + TasksSupplier(int numRemaining, Supplier> supplier) { this.numRemaining = numRemaining; this.supplier = supplier; } - public AbstractCompactionTask getTask() + public Collection getTasks() { return supplier.get(); } - public int compareTo(TaskSupplier o) + public int compareTo(TasksSupplier o) { return o.numRemaining - numRemaining; } @@ -130,12 +129,14 @@ boolean isEmpty() } protected final ColumnFamilyStore cfs; + protected final CompactionStrategyFactory strategyFactory; final DestinationRouter router; private int numTokenPartitions = -1; - AbstractStrategyHolder(ColumnFamilyStore cfs, DestinationRouter router) + AbstractStrategyHolder(ColumnFamilyStore cfs, CompactionStrategyFactory strategyFactory, DestinationRouter router) { this.cfs = cfs; + this.strategyFactory = strategyFactory; this.router = router; } @@ -166,11 +167,11 @@ public boolean managesSSTable(SSTableReader sstable) return managesRepairedGroup(sstable.isRepaired(), sstable.isPendingRepair(), sstable.isTransient()); } - public abstract AbstractCompactionStrategy getStrategyFor(SSTableReader sstable); + public abstract LegacyAbstractCompactionStrategy getStrategyFor(SSTableReader sstable); - public abstract Iterable allStrategies(); + public abstract Iterable allStrategies(); - public abstract Collection getBackgroundTaskSuppliers(int gcBefore); + public abstract Collection getBackgroundTaskSuppliers(int gcBefore); public abstract Collection getMaximalTasks(int gcBefore, boolean splitOutput); @@ -197,14 +198,8 @@ public abstract SSTableMultiWriter createSSTableMultiWriter(Descriptor descripto boolean isTransient, MetadataCollector collector, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker); - /** - * Return the directory index the given compaction strategy belongs to, or -1 - * if it's not held by this holder - */ - public abstract int getStrategyIndex(AbstractCompactionStrategy strategy); - public abstract boolean containsSSTable(SSTableReader sstable); } diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractTableOperation.java b/src/java/org/apache/cassandra/db/compaction/AbstractTableOperation.java new file mode 100644 index 000000000000..82ff5e4c49f9 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/AbstractTableOperation.java @@ -0,0 +1,242 @@ +/* + * 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.compaction; + +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.function.Predicate; + +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.TableMetadata; + +/** + * This is a base abstract implementing some default methods of {@link TableOperation}. + *

+ * In previous versions it used to be called CompactionInfo and CompactionInfo.Holder. + *

+ * This class implements serializable to allow structured info to be returned via JMX. + **/ +public abstract class AbstractTableOperation implements TableOperation +{ + private volatile boolean stopRequested = false; + private volatile StopTrigger trigger = StopTrigger.NONE; + + /** + * Interrupt the current operation if possible. + */ + public void stop() + { + stopRequested = true; + } + + /** + * Interrupt the current operation if possible and if the predicate is true. + * + * @param trigger cause of compaction interruption + */ + public void stop(StopTrigger trigger) + { + this.stopRequested = true; + if (!this.trigger.isFinal()) + this.trigger = trigger; + } + + /** + * @return true if the operation has received a request to be interrupted. + */ + public boolean isStopRequested() + { + return stopRequested || (isGlobal() && CompactionManager.instance.isGlobalCompactionPaused()); + } + + /** + * Return true if the predicate for the given sstables holds, or if the operation + * does not consider any sstables, in which case it will always return true (the + * default behaviour). + */ + public boolean shouldStop(Predicate predicate) + { + OperationProgress progress = getProgress(); + if (progress.sstables.isEmpty()) + { + return true; + } + return progress.sstables.stream().anyMatch(predicate); + } + + /** + * @return cause of compaction interruption. + */ + public StopTrigger trigger() + { + return trigger; + } + + /** + * The progress information for an operation, refer to the description of the class properties. + */ + public static final class OperationProgress implements Serializable, Progress + { + private static final long serialVersionUID = 3695381572726744816L; + + /** + * The table metadata + */ + private final TableMetadata metadata; + /** + * The type of operation + */ + private final OperationType operationType; + /** + * Normally the bytes processed so far by this operation, but depending on the unit it could mean something else, e.g. ranges or keys. + */ + private final long completed; + /** + * The total bytes that need to be processed, for example the size of the input files. Depending on the unit it could mean something else, e.g. ranges or keys. + */ + private final long total; + /** + * The unit for {@link this#completed} and for {@link this#total}. + */ + private final Unit unit; + /** + * A unique ID for this operation + */ + private final UUID operationId; + /** + * A set of SSTables participating in this operation + */ + private final ImmutableSet sstables; + + public OperationProgress(TableMetadata metadata, OperationType operationType, long bytesComplete, long totalBytes, UUID operationId, Collection sstables) + { + this(metadata, operationType, bytesComplete, totalBytes, Unit.BYTES, operationId, sstables); + } + + public OperationProgress(TableMetadata metadata, OperationType operationType, long completed, long total, Unit unit, UUID operationId, Collection sstables) + { + this.operationType = operationType; + this.completed = completed; + this.total = total; + this.metadata = metadata; + this.unit = unit; + this.operationId = operationId; + this.sstables = ImmutableSet.copyOf(sstables); + } + + /** + * @return A copy of this OperationProgress with updated progress. + */ + public OperationProgress forProgress(long complete, long total) + { + return new OperationProgress(metadata, operationType, complete, total, unit, operationId, sstables); + } + + public static OperationProgress withoutSSTables(TableMetadata metadata, OperationType tasktype, long completed, long total, AbstractTableOperation.Unit unit, UUID compactionId) + { + return new OperationProgress(metadata, tasktype, completed, total, unit, compactionId, ImmutableSet.of()); + } + + @Override + public Optional keyspace() + { + return metadata != null ? Optional.of(metadata.keyspace) : Optional.empty(); + } + + @Override + public Optional table() + { + return metadata != null ? Optional.of(metadata.name) : Optional.empty(); + } + + @Override + public TableMetadata metadata() + { + return metadata; + } + + @Override + public long completed() + { + return completed; + } + + @Override + public long total() + { + return total; + } + + @Override + public OperationType operationType() + { + return operationType; + } + + @Override + public UUID operationId() + { + return operationId; + } + + @Override + public Unit unit() + { + return unit; + } + + @Override + public Set sstables() + { + return sstables; + } + + public String toString() + { + StringBuilder buff = new StringBuilder(); + buff.append(String.format("%s(%s, %s / %s %s)", operationType, operationId, completed, total, unit)); + if (metadata != null) + { + buff.append(String.format("@%s(%s, %s)", metadata.id, metadata.keyspace, metadata.name)); + } + return buff.toString(); + } + + public Map asMap() + { + Map ret = new HashMap<>(8); + ret.put(ID, metadata != null ? metadata.id.toString() : ""); + ret.put(KEYSPACE, keyspace().orElse(null)); + ret.put(COLUMNFAMILY, table().orElse(null)); + ret.put(COMPLETED, Long.toString(completed)); + ret.put(TOTAL, Long.toString(total)); + ret.put(OPERATION_TYPE, operationType.toString()); + ret.put(UNIT, unit.toString()); + ret.put(OPERATION_ID, operationId == null ? "" : operationId.toString()); + return ret; + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/ActiveCompactions.java b/src/java/org/apache/cassandra/db/compaction/ActiveCompactions.java deleted file mode 100644 index 7b6b5bf1fe87..000000000000 --- a/src/java/org/apache/cassandra/db/compaction/ActiveCompactions.java +++ /dev/null @@ -1,75 +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.db.compaction; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.IdentityHashMap; -import java.util.List; -import java.util.Set; - -import org.apache.cassandra.io.sstable.format.SSTableReader; - -public class ActiveCompactions implements ActiveCompactionsTracker -{ - // a synchronized identity set of running tasks to their compaction info - private final Set compactions = Collections.synchronizedSet(Collections.newSetFromMap(new IdentityHashMap<>())); - - public List getCompactions() - { - return new ArrayList<>(compactions); - } - - public void beginCompaction(CompactionInfo.Holder ci) - { - compactions.add(ci); - } - - public void finishCompaction(CompactionInfo.Holder ci) - { - compactions.remove(ci); - CompactionManager.instance.getMetrics().bytesCompacted.inc(ci.getCompactionInfo().getTotal()); - CompactionManager.instance.getMetrics().totalCompactionsCompleted.mark(); - } - - /** - * Iterates over the active compactions and tries to find CompactionInfos with the given compactionType for the given sstable - * - * Number of entries in compactions should be small (< 10) but avoid calling in any time-sensitive context - */ - public Collection getCompactionsForSSTable(SSTableReader sstable, OperationType compactionType) - { - List toReturn = null; - synchronized (compactions) - { - for (CompactionInfo.Holder holder : compactions) - { - CompactionInfo compactionInfo = holder.getCompactionInfo(); - if (compactionInfo.getSSTables().contains(sstable) && compactionInfo.getTaskType() == compactionType) - { - if (toReturn == null) - toReturn = new ArrayList<>(); - toReturn.add(compactionInfo); - } - } - } - return toReturn; - } -} diff --git a/src/java/org/apache/cassandra/db/compaction/ActiveOperations.java b/src/java/org/apache/cassandra/db/compaction/ActiveOperations.java new file mode 100644 index 000000000000..d3da2caa5db9 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/ActiveOperations.java @@ -0,0 +1,93 @@ +/* + * 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.compaction; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Set; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.NonThrowingCloseable; + +public class ActiveOperations implements TableOperationObserver +{ + // The operations ordered by keyspace.table for all the operations that are currently in progress. + private static final Set operations = Collections.synchronizedSet(Collections.newSetFromMap(new IdentityHashMap<>())); + + /** + * @return all the table operations currently in progress. This is mostly compactions but it can include other + * operations too, basically any operation that calls {@link this#onOperationStart(TableOperation).} + */ + public List getTableOperations() + { + ImmutableList.Builder builder = ImmutableList.builder(); + builder.addAll(operations); + return builder.build(); + } + + @Override + public NonThrowingCloseable onOperationStart(TableOperation op) + { + operations.add(op); + return () -> { + operations.remove(op); + TableOperation.Progress progress = op.getProgress(); + CompactionManager.instance.getMetrics().bytesCompacted.inc(progress.total()); + CompactionManager.instance.getMetrics().totalCompactionsCompleted.mark(); + }; + } + + /** + * Iterates over the active operations and tries to find OperationProgresses with the given operation type for the given sstable + * + * Number of entries in operations should be small (< 10) but avoid calling in any time-sensitive context + */ + public Collection getOperationsForSSTable(SSTableReader sstable, OperationType operationType) + { + List toReturn = null; + + synchronized (operations) + { + for (TableOperation op : operations) + { + AbstractTableOperation.OperationProgress progress = op.getProgress(); + if (progress.sstables().contains(sstable) && progress.operationType() == operationType) + { + if (toReturn == null) + toReturn = new ArrayList<>(); + toReturn.add(progress); + } + } + } + return toReturn; + } + + /** + * @return true if given table operation is still active + */ + public boolean isActive(TableOperation op) + { + return getTableOperations().contains(op); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/ArenaSelector.java b/src/java/org/apache/cassandra/db/compaction/ArenaSelector.java new file mode 100644 index 000000000000..89157593f764 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/ArenaSelector.java @@ -0,0 +1,233 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.Iterables; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DiskBoundaries; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.io.sstable.format.SSTableReader; + +/** + * Arena selector, used by UnifiedCompactionStrategy to distribute SSTables to separate compaction arenas. + * + * This is used to: + * - ensure that sstables that should not be compacted together (e.g. repaired with unrepaired) are separated + * - ensure that each disk's sstables are compacted separately + * - implement compaction shards, subsections of the token space which compact separately for improved parallelism + * and compaction overheads. + */ +public class ArenaSelector implements Comparator +{ + private final EquivClassSplitter[] classSplitters; + final List shardBoundaries; + final DiskBoundaries diskBoundaries; + + public ArenaSelector(DiskBoundaries diskBoundaries, List shardBoundaries) + { + this.shardBoundaries = shardBoundaries; + this.diskBoundaries = diskBoundaries; + + ArrayList ret = new ArrayList<>(2); + + ret.add(RepairEquivClassSplitter.INSTANCE); + + if (diskBoundaries.getPositions() != null) + { + // The shard boundaries must also split on disks. Verify it. + assert new HashSet<>(shardBoundaries).containsAll(diskBoundaries.getPositions()); + } + else if (diskBoundaries.getNumBoundaries() > 1) + { + // We end up here if there are multiple disks, but not assigned according to token range. + ret.add(new DiskIndexEquivClassSplitter()); + } + + if (shardBoundaries.size() > 1) + ret.add(new ShardEquivClassSplitter()); + + classSplitters = ret.toArray(new EquivClassSplitter[0]); + } + + @Override + public int compare(SSTableReader o1, SSTableReader o2) + { + int res = 0; + for (int i = 0; res == 0 && i < classSplitters.length; i++) + res = classSplitters[i].compare(o1, o2); + return res; + } + + public String name(SSTableReader t) + { + return Arrays.stream(classSplitters) + .map(e -> e.name(t)) + .collect(Collectors.joining("-")); + } + + /** + * Returns the shard where this key belongs. Shards are given by their end boundaries (i.e. shard 0 covers the space + * between minimum and shardBoundaries[0], shard 1 is is between shardBoundaries[0] and shardBoundaries[1]), thus + * finding the index of the first bigger boundary gives the index of the covering shard. + */ + public int shardFor(DecoratedKey key) + { + return shardFor(key, shardBoundaries); + } + + public static int shardFor(DecoratedKey key, List shardBoundaries) + { + int pos = Collections.binarySearch(shardBoundaries, key); + assert pos < 0; // boundaries are .minkeybound and .maxkeybound so they should never be equal to a DecoratedKey + return -pos - 1; + } + + public static int shardsSpanned(SSTableReader rdr, List shardBoundaries) + { + if (shardBoundaries.size() <= 1) + return 1; + int startIdx = shardFor(rdr.getFirst(), shardBoundaries); + DecoratedKey last = rdr.getLast(); + if (last.compareTo(shardBoundaries.get(startIdx)) < 0) + return 1; // quick path, end boundary is in the same shard + return shardFor(last, shardBoundaries) - startIdx + 1; + } + + public long shardAdjustedSize(SSTableReader rdr) + { + return shardAdjustedSize(rdr, shardBoundaries); + } + + public static long shardAdjustedSize(SSTableReader rdr, List shardBoundaries) + { + // This may need to duplicate the above to avoid the division in the happy path + return rdr.onDiskLength() / shardsSpanned(rdr, shardBoundaries); + } + + public static Set sstablesFor(int boundaryIndex, List shardBoundaries, Set sstables) + { + assert boundaryIndex < shardBoundaries.size(); + return sstables.stream() + .filter(sstable -> shardFor(sstable.getFirst(), shardBoundaries) <= boundaryIndex && shardFor(sstable.getLast(), shardBoundaries) >= boundaryIndex) + .collect(Collectors.toSet()); + } + + public int compareByShardAdjustedSize(SSTableReader a, SSTableReader b) + { + return Long.compare(shardAdjustedSize(a), shardAdjustedSize(b)); + } + + /** + * An equivalence class is a function that compares two sstables and returns 0 when they fall in the same class. + * For example, the repair status or disk index may define equivalence classes. See the concrete equivalence classes below. + */ + private interface EquivClassSplitter extends Comparator { + + @Override + int compare(SSTableReader a, SSTableReader b); + + /** Return a name that describes the equivalence class */ + String name(SSTableReader ssTableReader); + } + + /** + * Split sstables by their repair state: repaired, unrepaired, pending repair with a specific UUID (one group per pending repair). + */ + private static final class RepairEquivClassSplitter implements EquivClassSplitter + { + public static final EquivClassSplitter INSTANCE = new RepairEquivClassSplitter(); + + @Override + public int compare(SSTableReader a, SSTableReader b) + { + // This is the same as name(a).compareTo(name(b)) + int af = a.isRepaired() ? 1 : !a.isPendingRepair() ? 2 : 0; + int bf = b.isRepaired() ? 1 : !b.isPendingRepair() ? 2 : 0; + if (af != 0 || bf != 0) + return Integer.compare(af, bf); + return a.getPendingRepair().compareTo(b.getPendingRepair()); + } + + @Override + public String name(SSTableReader ssTableReader) + { + if (ssTableReader.isRepaired()) + return "repaired"; + else if (!ssTableReader.isPendingRepair()) + return "unrepaired"; + else + return "pending_repair_" + ssTableReader.getPendingRepair(); + } + } + + /** + * Split sstables by their shard. If the data set size is larger than the shard size in the compaction options, + * then we create an equivalence class based by shard. Each sstable ends up in a shard based on their first + * key. Each shard is calculated by splitting the local token ranges into a number of shards, where the number + * of shards is calculated as ceil(data_size / shard size); + * + * Shard boundaries also split the sstables that reside on different disks. + */ + private final class ShardEquivClassSplitter implements EquivClassSplitter + { + @Override + public int compare(SSTableReader a, SSTableReader b) + { + return Integer.compare(shardFor(a.getFirst()), shardFor(b.getFirst())); + } + + @Override + public String name(SSTableReader ssTableReader) + { + return "shard_" + shardFor(ssTableReader.getFirst()); + } + } + + /** + * Group sstables by their disk index. + */ + private final class DiskIndexEquivClassSplitter implements EquivClassSplitter + { + @Override + public int compare(SSTableReader a, SSTableReader b) + { + return Integer.compare(diskBoundaries.getDiskIndexFromKey(a), diskBoundaries.getDiskIndexFromKey(b)); + } + + @Override + public String name(SSTableReader ssTableReader) + { + return "disk_" + diskBoundaries.getDiskIndexFromKey(ssTableReader); + } + } + + + // TODO - missing equivalence classes: + + // - by time window to emulate TWCS, in this case only the latest shard will use size based buckets, the older + // shards will get major compactions +} diff --git a/src/java/org/apache/cassandra/db/compaction/BackgroundCompactions.java b/src/java/org/apache/cassandra/db/compaction/BackgroundCompactions.java new file mode 100644 index 000000000000..ef760f85df66 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/BackgroundCompactions.java @@ -0,0 +1,310 @@ +/* + * 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.compaction; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +import com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.ExpMovingAverage; +import org.apache.cassandra.utils.MovingAverage; + +/** + * A class for grouping the background compactions picked by a strategy, either pending or in progress. + * + * A compaction strategy has a {@link BackgroundCompactions} object as part of its state. Each + * {@link LegacyAbstractCompactionStrategy} instance has its {@link BackgroundCompactions}, and their lifespans are the + * same. In the case of {@link UnifiedCompactionStrategy} the new strategy instance inherits + * {@link BackgroundCompactions} from its predecessor. + */ +public class BackgroundCompactions +{ + private static final Logger logger = LoggerFactory.getLogger(BackgroundCompactions.class); + + /** The table metadata */ + private final TableMetadata metadata; + + /** The compaction aggregates with either pending or ongoing compactions, or both. This is a private map + * whose access needs to be synchronized. */ + private final TreeMap aggregatesMap; + + /** + * The current list of compaction aggregates, this list must be recreated every time the aggregates + * map is changed. + * + * We publish aggregates to a separate variable instead of calling {@code aggregatesMap.values()} so that reads + * that race with updates always observe a consistent snapshot. + */ + private volatile List aggregates; + + /** The ongoing compactions grouped by unique operation ID. */ + private final ConcurrentHashMap compactions = new ConcurrentHashMap<>(); + + /** + * Rate of progress (per thread) of recent compactions for the CFS. Used by the UnifiedCompactionStrategy to + * limit the number of running compactions to no more than what is sufficient to saturate the throughput limit. + * This needs to be a longer-running average to ensure that the rate limiter stalling a new thread can't cause + * the compaction rate to temporarily drop to levels that permit an extra thread. + */ + MovingAverage compactionRate = ExpMovingAverage.decayBy1000(); + + BackgroundCompactions(ColumnFamilyStore cfs) + { + this.metadata = cfs.metadata(); + this.aggregatesMap = new TreeMap<>(); + this.aggregates = ImmutableList.of(); + } + + /** + * Updates the list of pending compactions, while preserving the set of running ones. This is done + * by creating new aggregates with the pending aggregates but adding any existing aggregates with + * compactions in progress. If there is a matching pending aggregate then the existing compactions + * are transferred to it, otherwise the old aggregate is stripped of its pending compactios and then + * it is kept with the compactions in progress only. + * + * @param pending compaction aggregates with pending compactions + */ + synchronized void setPending(CompactionStrategy strategy, Collection pending) + { + if (pending == null) + throw new IllegalArgumentException("argument cannot be null"); + + if (logger.isTraceEnabled()) + logger.trace("Resetting pending aggregates for strategy {}/{}, received {} new aggregates", + strategy.getName(), strategy.hashCode(), pending.size()); + + // First remove the existing aggregates + aggregatesMap.clear(); + + // Then add all the pending aggregates + for (CompactionAggregate aggregate : pending) + { + CompactionAggregate prev = aggregatesMap.put(aggregate.getKey(), aggregate); + if (logger.isTraceEnabled()) + logger.trace("Adding new pending aggregate: {}", aggregate); + + if (prev != null) + throw new IllegalArgumentException("Received pending aggregates with non unique keys: " + prev.getKey()); + } + + // Then add the old aggregates but only if they have ongoing compactions + for (CompactionAggregate oldAggregate : this.aggregates) + { + Collection compacting = oldAggregate.getInProgress(); + if (compacting.isEmpty()) + { + if (logger.isTraceEnabled()) + logger.trace("Existing aggregate {} has no in progress compactions, removing it", oldAggregate); + + continue; + } + + // See if we have a matching aggregate in the pending aggregates, if so add all the existing compactions to it + // otherwise strip the pending and selected compactions from the old one and keep it only with the compactions in progress + CompactionAggregate newAggregate; + CompactionAggregate matchingAggregate = oldAggregate.getMatching(aggregatesMap); + if (matchingAggregate != null) + { + // add the old compactions to the new aggregate + // the key will change slightly for STCS so remove it before adding it again + aggregatesMap.remove(matchingAggregate.getKey()); + newAggregate = matchingAggregate.withAdditionalCompactions(compacting); + + if (logger.isTraceEnabled()) + logger.trace("Removed matching aggregate {}", matchingAggregate); + } + else + { + // keep the old aggregate but only with the compactions already in progress and not yet completed + newAggregate = oldAggregate.withOnlyTheseCompactions(compacting); + + if (logger.isTraceEnabled()) + logger.trace("Keeping old aggregate but only with compactions {}", oldAggregate); + } + + if (logger.isTraceEnabled()) + logger.trace("Adding new aggregate with previous compactions {}", newAggregate); + + aggregatesMap.put(newAggregate.getKey(), newAggregate); + } + + // Publish the new aggregates + this.aggregates = ImmutableList.copyOf(aggregatesMap.values()); + + CompactionLogger compactionLogger = strategy.getCompactionLogger(); + if (compactionLogger != null && compactionLogger.enabled()) + { + // compactionLogger.statistics(strategy, "pending", getStatistics()); // too much noise + compactionLogger.pending(strategy, getEstimatedRemainingTasks()); + } + } + + void setSubmitted(CompactionStrategy strategy, UUID id, CompactionAggregate aggregate) + { + if (id == null || aggregate == null) + throw new IllegalArgumentException("arguments cannot be null"); + + logger.debug("Submitting background compaction {}", id); + CompactionPick compaction = aggregate.getSelected(); + + CompactionPick prev = compactions.put(id, compaction); + if (prev != null) + throw new IllegalArgumentException("Found existing compaction with same id: " + id); + + compaction.setSubmitted(id); + + synchronized (this) + { + CompactionAggregate existingAggregate = aggregate.getMatching(aggregatesMap); + boolean aggregatesMapChanged = false; + + if (existingAggregate == null) + { + if (logger.isTraceEnabled()) + logger.trace("Could not find aggregate for compaction using the one passed in: {}", aggregate); + + aggregatesMapChanged = true; + aggregatesMap.put(aggregate.getKey(), aggregate); + } + else + { + if (logger.isTraceEnabled()) + logger.trace("Found aggregate for compaction: {}", existingAggregate); + + if (!existingAggregate.getActive().contains(compaction)) + { + // add the compaction just submitted to the aggregate that was found but because for STCS its + // key may change slightly, first remove it + aggregatesMapChanged = true; + aggregatesMap.remove(existingAggregate.getKey()); + CompactionAggregate newAggregate = existingAggregate.withAdditionalCompactions(ImmutableList.of(compaction)); + aggregatesMap.put(newAggregate.getKey(), newAggregate); + + if (logger.isTraceEnabled()) + logger.trace("Added compaction to existing aggregate: {} -> {}", existingAggregate, newAggregate); + } + else + { + if (logger.isTraceEnabled()) + logger.trace("Existing aggregate {} already had compaction", existingAggregate); + } + } + + // Publish the new aggregates if needed + if (aggregatesMapChanged) + this.aggregates = ImmutableList.copyOf(aggregatesMap.values()); + } + + CompactionLogger compactionLogger = strategy.getCompactionLogger(); + if (compactionLogger != null && compactionLogger.enabled()) + compactionLogger.statistics(strategy, "submitted", getStatistics(strategy)); + } + + public void onInProgress(CompactionProgress progress) + { + if (progress == null) + throw new IllegalArgumentException("argument cannot be null"); + + updateCompactionRate(progress); + + CompactionPick compaction = compactions.computeIfAbsent(progress.operationId(), + uuid -> CompactionPick.create(-1, progress.inSSTables())); + + logger.debug("Setting background compaction {} as in progress", progress.operationId()); + compaction.setProgress(progress); + } + + public void onCompleted(CompactionStrategy strategy, UUID id) + { + if (id == null) + throw new IllegalArgumentException("argument cannot be null"); + + logger.debug("Removing compaction {}", id); + + // log the statistics before completing the compaction so that we see the stats for the + // compaction that just completed + CompactionLogger compactionLogger = strategy.getCompactionLogger(); + if (compactionLogger != null && compactionLogger.enabled()) + compactionLogger.statistics(strategy, "completed", getStatistics(strategy)); + + CompactionPick completed = compactions.remove(id); + CompactionProgress progress = completed.progress; + updateCompactionRate(progress); + + if (completed != null) + completed.setCompleted(); + + // We rely on setPending() to refresh the aggregates again even though in some cases it may not be + // called immediately (e.g. compactions disabled) + } + + private void updateCompactionRate(CompactionProgress progress) + { + if (progress != null && progress.durationInNanos() > 0 && progress.outputDiskSize() > 0) + compactionRate.update(progress.outputDiskSize() * 1.e9 / progress.durationInNanos()); + } + + public Collection getAggregates() + { + return aggregates; + } + + /** + * @return the number of background compactions estimated to still be needed + */ + public int getEstimatedRemainingTasks() + { + return CompactionAggregate.numEstimatedCompactions(aggregates); + } + + /** + * @return the compactions currently in progress + */ + public Collection getCompactionsInProgress() + { + return Collections.unmodifiableCollection(compactions.values()); + } + + /** + * @return the total number of background compactions, pending or in progress + */ + public int getTotalCompactions() + { + return compactions.size() + getEstimatedRemainingTasks(); + } + + /** + * Return the compaction statistics for this strategy. + * + * @return statistics about this compaction strategy. + */ + public CompactionStrategyStatistics getStatistics(CompactionStrategy strategy) + { + return CompactionAggregate.getStatistics(metadata, strategy, aggregates); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/CleanupTask.java b/src/java/org/apache/cassandra/db/compaction/CleanupTask.java new file mode 100644 index 000000000000..8daf43505035 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CleanupTask.java @@ -0,0 +1,83 @@ +/* + * 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.compaction; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.repair.consistent.admin.CleanupSummary; +import org.apache.cassandra.utils.Pair; + +public class CleanupTask +{ + private static final Logger logger = LoggerFactory.getLogger(CleanupTask.class); + + private final ColumnFamilyStore cfs; + private final List> tasks; + + public CleanupTask(ColumnFamilyStore cfs, List> tasks) + { + this.cfs = cfs; + this.tasks = tasks; + } + + public CleanupSummary cleanup() + { + Set successful = new HashSet<>(); + Set unsuccessful = new HashSet<>(); + for (Pair pair : tasks) + { + UUID session = pair.left; + RepairFinishedCompactionTask task = pair.right; + + if (task != null) + { + try + { + task.run(); + successful.add(session); + } + catch (Throwable t) + { + t = task.transaction.abort(t); + logger.error("Failed cleaning up " + session, t); + unsuccessful.add(session); + } + } + else + { + unsuccessful.add(session); + } + } + return new CleanupSummary(cfs, successful, unsuccessful); + } + + public Throwable abort(Throwable accumulate) + { + for (Pair pair : tasks) + accumulate = pair.right.transaction.abort(accumulate); + return accumulate; + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionAggregate.java b/src/java/org/apache/cassandra/db/compaction/CompactionAggregate.java new file mode 100644 index 000000000000..6c7bb9f73e1d --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionAggregate.java @@ -0,0 +1,892 @@ +/* + * 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.compaction; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.Set; +import java.util.SortedMap; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.FBUtilities; + +/** + * A compaction aggregate is either a level in {@link LeveledCompactionStrategy} or a tier (bucket) in other + * compaction strategies. + *

+ * It contains a list of {@link CompactionPick}, which are the compactions either in progress or pending. + * It also contains a selected {@link CompactionPick}, which is a compaction about to be submitted. The submitted + * compaction is also part of the compactions. Lastly, it contains a set of all the sstables in this aggregate, + * regardless of whether they need compaction. + */ +public abstract class CompactionAggregate +{ + private static final Logger logger = LoggerFactory.getLogger(CompactionAggregate.class); + + /** The unique key that identifies this aggregate. */ + final Key key; + + /** The sstables in this aggregate, whether they are compaction candidates or not */ + final Set sstables; + + /** The compaction that was selected for this aggregate when it was created. It is also part of {@link this#compactions}. */ + final CompactionPick selected; + + /** The compactions that are part of this aggregate, they could be pending or in progress. */ + final LinkedHashSet compactions; + + CompactionAggregate(Key key, Iterable sstables, CompactionPick selected, Iterable pending) + { + if (sstables == null || selected == null || pending == null) + throw new IllegalArgumentException("Arguments cannot be null"); + + this.key = key; + this.sstables = new HashSet<>(); sstables.forEach(this.sstables::add); + this.selected = selected; + + // Here we want to keep the iteration order since normally pending compactions are ordered by a strategy + // and the selected compaction should be the first one + this.compactions = new LinkedHashSet<>(); + if (!selected.isEmpty()) + compactions.add(selected); + + for (CompactionPick p : pending) + { + if (p == null || p.isEmpty()) + throw new IllegalArgumentException("Pending compactions should be valid compactions"); + + compactions.add(p); + } + } + + public CompactionPick getSelected() + { + return selected; + } + + /** + * @return the total sstable size for all the compaction picks that are either pending or still in progress + */ + public long getPendingBytes() + { + long ret = 0; + for (CompactionPick comp : compactions) + { + if (comp.id == null) + ret += comp.totSizeInBytes; + } + return ret; + } + + /** + * @return compactions that have not yet been submitted (no compaction id). + */ + public List getPending() + { + List ret = new ArrayList<>(compactions.size()); + for (CompactionPick comp : compactions) + { + if (comp.id == null) + ret.add(comp); + } + + return ret; + } + + /** + * @return compactions that have already been submitted (compaction id is available) and haven't completed yet + */ + public List getInProgress() + { + List ret = new ArrayList<>(compactions.size()); + for (CompactionPick comp : compactions) + { + if (comp.id != null && !comp.completed) + ret.add(comp); + } + + return ret; + } + + /** + * @return all the compactions we have + */ + public List getActive() + { + return new ArrayList<>(compactions); + } + + /** + * @return true if this aggregate has no compactions + */ + public boolean isEmpty() + { + return compactions.isEmpty(); + } + + /** + * Merge the pending compactions and the compactions in progress to create some aggregated statistics. + * + * @return the statistics for this compaction aggregate, see {@link CompactionAggregateStatistics}. + */ + public abstract CompactionAggregateStatistics getStatistics(); + + /** + * Calculates basic compaction statistics, common for all types of {@link CompactionAggregate}s. + * + * @param trackHotness Indicates whether aggregate (tier/bucket) hotness is relevant and should be calculated. + * If this is {@code false}, a default value of {@link Double#NaN} will be used to indicate + * that hotness hasn't been calculated. + * + * @return a new {@link CompactionAggregateStatistics} instance, containing all the common statistics for the + * different types of {@link CompactionAggregate}s (see above for the caveat about hotness). + */ + CompactionAggregateStatistics getCommonStatistics(boolean trackHotness) + { + int numCompactions = 0; + int numCompactionsInProgress = 0; + int numCandidateSSTables = 0; + int numCompactingSSTables = 0; + int numExpiredSSTables = 0; + long tot = 0; + long expiredTot = 0; + double hotness = trackHotness ? 0.0 : Double.NaN; + long read = 0; + long written = 0; + long durationNanos = 0; + + for (CompactionPick compaction : compactions) + { + if (compaction.completed) + continue; + + numCompactions++; + numCandidateSSTables += compaction.sstables.size(); + numExpiredSSTables += compaction.expired.size(); + tot += compaction.sstables.stream().mapToLong(SSTableReader::uncompressedLength).reduce(0L, Long::sum); + expiredTot += compaction.expired.stream().mapToLong(SSTableReader::uncompressedLength).reduce(0L, Long::sum); + if (trackHotness) + hotness += compaction.hotness; + + if (compaction.id != null) + { + numCompactionsInProgress++; + numCompactingSSTables += compaction.sstables.size(); + } + + if (compaction.progress != null) + { + read += compaction.progress.uncompressedBytesRead(); + written += compaction.progress.uncompressedBytesWritten(); + durationNanos += compaction.progress.durationInNanos(); + } + } + + return new CompactionAggregateStatistics(numCompactions, + numCompactionsInProgress, + sstables.size(), + numExpiredSSTables, + numCandidateSSTables, + numCompactingSSTables, + getTotSizeBytes(sstables), + tot, + expiredTot, + read, + written, + durationNanos, + hotness); + } + + /** + * @return the number of estimated compactions that are still pending. + */ + public int numEstimatedCompactions() + { + return getPending().size(); + } + + /** + * @return a key that ensures the uniqueness of an aggregate but also that allows identify future identical aggregates, + * e.g. when an aggregate is merged with an older aggregate that has still ongoing compactions like a level + * in LCS or a bucket in the unified strategy or STCS or a time window in TWCS + */ + public Key getKey() + { + return key; + } + + /** + * Return a matching aggregate from the map passed in or null. Normally this is just a matter of finding + * the key in the map but for STCS we need to look at the possible min and maximum average sizes and so + * {@link SizeTiered} overrides this method. + * + * @param others a map of other aggregates + * + * @return an aggregate with the same key or null + */ + @Nullable CompactionAggregate getMatching(NavigableMap others) + { + return others.get(getKey()); + } + + /** + * Create a copy of this aggregate with the new parameters + * + * @return a deep copy of this aggregate + */ + protected abstract CompactionAggregate clone(Iterable sstables, CompactionPick selected, Iterable compactions); + + /** + * Add expired sstables to the selected compaction pick and return a new compaction aggregate. + */ + CompactionAggregate withExpired(Collection expired) + { + return clone(Iterables.concat(sstables, expired), selected.withExpiredSSTables(expired), compactions); + } + + /** + * Add existing compactions to our own compactions and return a new compaction aggregate + */ + public CompactionAggregate withAdditionalCompactions(Collection comps) + { + List sstables = comps.stream().flatMap(comp -> comp.sstables.stream()).collect(Collectors.toList()); + return clone(Iterables.concat(this.sstables, sstables), selected, Iterables.concat(compactions, comps)); + } + + /** + * Only keep the compactions passed in, strip everything else. + */ + public CompactionAggregate withOnlyTheseCompactions(Collection comps) + { + List sstables = comps.stream().flatMap(comp -> comp.sstables.stream()).collect(Collectors.toList()); + return clone(sstables, CompactionPick.EMPTY, comps); + } + + @Override + public int hashCode() + { + return Objects.hash(sstables, selected, compactions); + } + + @Override + public boolean equals(Object obj) + { + if (obj == this) + return true; + + if (!(obj instanceof CompactionAggregate)) + return false; + + CompactionAggregate that = (CompactionAggregate) obj; + return sstables.equals(that.sstables) && + selected.equals(that.selected) && + compactions.equals(that.compactions); + } + + /** + * Contains information about a levelled compaction aggregate, this is equivalent to a level in {@link LeveledCompactionStrategy}. + */ + public static final class Leveled extends CompactionAggregate + { + /** The current level number */ + final int level; + + /** The next level number */ + final int nextLevel; + + /** The score of this level as defined in {@link LeveledCompactionStrategy}. */ + final double score; + + /** The maximum size of each output sstable that will be produced by compaction, Long.MAX_VALUE if no maximum exists */ + final long maxSSTableBytes; + + /** + * How many more compactions this level is expected to perform. This is required because for LCS we cannot + * easily identify candidate sstables to put into the pending picks. + */ + final int pendingCompactions; + + /** The fanout size */ + final int fanout; + + Leveled(Iterable sstables, + CompactionPick selected, + Iterable compactions, + int level, + int nextLevel, + double score, + long maxSSTableBytes, + int pendingCompactions, + int fanout) + { + super(new Key(level), sstables, selected, compactions); + + this.level = level; + this.nextLevel = nextLevel; + this.score = score; + this.maxSSTableBytes = maxSSTableBytes; + this.pendingCompactions = pendingCompactions; + this.fanout = fanout; + } + + @Override + protected CompactionAggregate clone(Iterable sstables, CompactionPick selected, Iterable compactions) + { + return new Leveled(sstables, selected, compactions, level, nextLevel, score, maxSSTableBytes, pendingCompactions, fanout); + } + + @Override + public CompactionAggregateStatistics getStatistics() + { + CompactionAggregateStatistics stats = getCommonStatistics(false); + + long readLevel = 0L; + + for (CompactionPick compaction : compactions) + if (!compaction.completed && compaction.progress != null) + readLevel += compaction.progress.uncompressedBytesRead(level); + + return new LeveledCompactionStatistics(stats, level, score, pendingCompactions, readLevel); + } + + @Override + public int numEstimatedCompactions() + { + return pendingCompactions; + } + + @Override + public boolean isEmpty() + { + return super.isEmpty() && pendingCompactions == 0; + } + + @Override + public String toString() + { + return String.format("Level %d with %d sstables, %d compactions and %d pending", level, sstables.size(), compactions.size(), pendingCompactions); + } + } + + /** + * Create a level where we have a compaction candidate. + */ + static CompactionAggregate.Leveled createLeveled(Collection all, + Collection candidates, + int pendingCompactions, + long maxSSTableBytes, + int level, + int nextLevel, + double score, + int fanout) + { + return new Leveled(all, + CompactionPick.create(level, candidates), + ImmutableList.of(), + level, + nextLevel, + score, + maxSSTableBytes, + pendingCompactions, + fanout); + } + + /** + * Create a level when we only have estimated tasks. + */ + static CompactionAggregate.Leveled createLeveled(Collection all, + int pendingCompactions, + long maxSSTableBytes, + int level, + double score, + int fanout) + { + return new Leveled(all, + CompactionPick.EMPTY, + ImmutableList.of(), + level, + level + 1, + score, + maxSSTableBytes, + pendingCompactions, + fanout); + } + + /** + * Create a leveled aggregate when LCS is doing STCS on level 0 + */ + static CompactionAggregate.Leveled createLeveledForSTCS(Collection all, + CompactionPick pick, + int pendingCompactions, + double score, + int fanout) + { + return new Leveled(all, + pick, + ImmutableList.of(), + 0, + 0, + score, + Long.MAX_VALUE, + pendingCompactions, + fanout); + } + + /** + * Contains information about a size-tiered compaction aggregate, this is equivalent to a bucket in {@link SizeTieredCompactionStrategy}. + */ + public static final class SizeTiered extends CompactionAggregate + { + /** The total read hotness of the sstables in this tier, as defined by {@link SSTableReader#hotness()} */ + final double hotness; + + /** The average on disk size in bytes of the sstables in this tier */ + final long avgSizeBytes; + + /** The minimum on disk size in bytes for this tier, this is normally the avg size times the STCS bucket low and it is + * used to find compacting aggregates that are on the same tier. */ + final long minSizeBytes; + + /** The maximum on disk size in bytes for this tier, this is normally the avg size times the STCS bucket high and it is + * used to find compacting aggregates that are on the same tier. */ + final long maxSizeBytes; + + SizeTiered(Iterable sstables, + CompactionPick selected, + Iterable pending, + double hotness, + long avgSizeBytes, + long minSizeBytes, + long maxSizeBytes) + { + super(new Key(avgSizeBytes), sstables, selected, pending); + + this.hotness = hotness; + this.avgSizeBytes = avgSizeBytes; + this.minSizeBytes = minSizeBytes; + this.maxSizeBytes = maxSizeBytes; + } + + @Override + protected CompactionAggregate clone(Iterable sstables, CompactionPick selected, Iterable compactions) + { + return new SizeTiered(sstables, selected, compactions, getTotHotness(sstables), getAvgSizeBytes(sstables), minSizeBytes, maxSizeBytes); + } + + @Override + public CompactionAggregateStatistics getStatistics() + { + CompactionAggregateStatistics stats = getCommonStatistics(true); + + return new SizeTieredCompactionStatistics(stats, avgSizeBytes); + } + + @Override + @Nullable CompactionAggregate getMatching(NavigableMap others) + { + SortedMap subMap = others.subMap(new Key(minSizeBytes), new Key(maxSizeBytes)); + if (subMap.isEmpty()) + { + if (logger.isTraceEnabled()) + logger.trace("Found no matching aggregate for {}", + FBUtilities.prettyPrintMemory(avgSizeBytes)); + + return null; + } + + if (logger.isTraceEnabled()) + logger.trace("Found {} matching aggregates for {}", + subMap.size(), + FBUtilities.prettyPrintMemory(avgSizeBytes)); + + Key closest = null; + long minDiff = 0; + for (Key m : subMap.keySet()) + { + long diff = Math.abs(m.index - avgSizeBytes); + if (closest == null || diff < minDiff) + { + closest = m; + minDiff = diff; + } + } + + if (logger.isTraceEnabled()) + logger.trace("Using closest matching aggregate for {}: {}", + FBUtilities.prettyPrintMemory(avgSizeBytes), + FBUtilities.prettyPrintMemory(closest != null ? closest.index : -1)); + + return others.get(closest); + } + + @Override + public String toString() + { + return String.format("Size tiered %s/%s/%s with %d sstables, %d compactions", + FBUtilities.prettyPrintMemory(minSizeBytes), + FBUtilities.prettyPrintMemory(avgSizeBytes), + FBUtilities.prettyPrintMemory(maxSizeBytes), + sstables.size(), + compactions.size()); + } + } + + static CompactionAggregate createSizeTiered(Collection all, + CompactionPick selected, + List pending, + double hotness, + long avgSizeBytes, + long minSizeBytes, + long maxSizeBytes) + { + return new SizeTiered(all, selected, pending, hotness, avgSizeBytes, minSizeBytes, maxSizeBytes); + } + + /** + * Contains information about a size-tiered compaction aggregate, this is equivalent to a bucket in {@link SizeTieredCompactionStrategy}. + */ + public static final class TimeTiered extends CompactionAggregate + { + /** The timestamp of this aggregate */ + final long timestamp; + + TimeTiered(Iterable sstables, CompactionPick selected, Iterable pending, long timestamp) + { + super(new Key(timestamp), sstables, selected, pending); + this.timestamp = timestamp; + } + + @Override + protected CompactionAggregate clone(Iterable sstables, CompactionPick selected, Iterable compactions) + { + return new TimeTiered(sstables, selected, compactions, timestamp); + } + + @Override + public CompactionAggregateStatistics getStatistics() + { + CompactionAggregateStatistics stats = getCommonStatistics(true); + return new TimeTieredCompactionStatistics(stats, timestamp); + } + + @Override + public String toString() + { + return String.format("Time tiered %d with %d sstables, %d compactions", timestamp, sstables.size(), compactions.size()); + } + } + + static CompactionAggregate createTimeTiered(Collection sstables, long timestamp) + { + return new TimeTiered(sstables, CompactionPick.create(timestamp, sstables), ImmutableList.of(), timestamp); + } + + static CompactionAggregate createTimeTiered(Collection sstables, CompactionPick selected, List pending, long timestamp) + { + return new TimeTiered(sstables, selected, pending, timestamp); + } + + public static final class UnifiedAggregate extends CompactionAggregate + { + /** The shard to which this bucket belongs */ + private final UnifiedCompactionStrategy.Shard shard; + + /** The bucket generated by the compaction strategy */ + private final UnifiedCompactionStrategy.Bucket bucket; + + UnifiedAggregate(Iterable sstables, + CompactionPick selected, + Iterable pending, + UnifiedCompactionStrategy.Shard shard, + UnifiedCompactionStrategy.Bucket bucket) + { + super(new ShardedKey(shard, bucket.index), sstables, selected, pending); + this.shard = shard; + this.bucket = bucket; + } + + public UnifiedCompactionStrategy.Shard getShard() + { + return shard; + } + + @Override + public CompactionAggregateStatistics getStatistics() + { + CompactionAggregateStatistics stats = getCommonStatistics(false); + + return new UnifiedCompactionStatistics(stats, + bucket.index, + bucket.survivalFactor, + bucket.scalingParameter, + bucket.threshold, + bucket.fanout, + bucket.min, + bucket.max, + shard.name()); + } + + @Override + protected CompactionAggregate clone(Iterable sstables, CompactionPick selected, Iterable compactions) + { + return new UnifiedAggregate(sstables, selected, compactions, shard, bucket); + } + + int bucketIndex() + { + return bucket.index; + } + + @Override + public String toString() + { + return String.format("Unified shard %s bucket %d with %d sstables and %d compactions", + shard.name(), + bucket.index, + sstables.size(), + compactions.size()); + } + + @Override + public boolean equals(Object obj) + { + if (obj == this) + return true; + + if (!(obj instanceof UnifiedAggregate)) + return false; + + UnifiedAggregate that = (UnifiedAggregate) obj; + return sstables.equals(that.sstables) && + selected.equals(that.selected) && + compactions.equals(that.compactions) && + bucket.equals(that.bucket) && + shard.equals(that.shard); + } + + @Override + public int hashCode() + { + return Objects.hash(sstables, selected, compactions, bucket, shard); + } + } + + static UnifiedAggregate createUnified(Collection sstables, + CompactionPick selected, + Iterable pending, + UnifiedCompactionStrategy.Shard shard, + UnifiedCompactionStrategy.Bucket bucket) + { + return new UnifiedAggregate(sstables, selected, pending, shard, bucket); + } + + + + /** An aggregate that is created for a compaction issued only to drop tombstones */ + public static final class TombstoneAggregate extends CompactionAggregate + { + TombstoneAggregate(Iterable sstables, CompactionPick selected, Iterable pending) + { + super(new Key(-1), sstables, selected, pending); + } + + @Override + protected CompactionAggregate clone(Iterable sstables, CompactionPick selected, Iterable compactions) + { + return new TombstoneAggregate(sstables, selected, compactions); + } + + @Override + public CompactionAggregateStatistics getStatistics() + { + return getCommonStatistics(false); + } + + @Override + public String toString() + { + return String.format("Tombstones with %d sstables, %d compactions", sstables.size(), compactions.size()); + } + } + + static CompactionAggregate createForTombstones(SSTableReader sstable) + { + List sstables = ImmutableList.of(sstable); + CompactionPick comp = CompactionPick.create(-1, sstables); + return new TombstoneAggregate(sstables, comp, ImmutableList.of()); + } + + /** + * A key suitable for a strategy that has no shards, that is a legacy strategy that is + * managed by CompactionStrategyManager. + */ + public static class Key implements Comparable + { + protected final long index; + + Key(long index) + { + this.index = index; + } + + @Override + public int compareTo(Key key) + { + return Long.compare(index, key.index); + } + } + + /** + * A key suitable for a strategy using shards, first it compares by shard, and then by bucket index. + */ + private static final class ShardedKey extends Key + { + private final UnifiedCompactionStrategy.Shard shard; + + ShardedKey(UnifiedCompactionStrategy.Shard shard, long index) + { + super(index); + this.shard = shard; + } + + @Override + public int compareTo(Key key) + { + if (key instanceof ShardedKey) + { + ShardedKey shardedKey = (ShardedKey) key; + + int ret = shard.compareTo(shardedKey.shard); + if (ret != 0) + return ret; + } + + // either not sharded or same shard + return Long.compare(index, key.index); + } + } + + /** + * Return the compaction statistics for this strategy and list of compactions that are either pending or in progress. + * + * @param aggregates the compaction aggregates + * + * @return the statistics about this compactions + */ + static CompactionStrategyStatistics getStatistics(TableMetadata metadata, + CompactionStrategy strategy, + Collection aggregates) + { + List statistics = new ArrayList<>(aggregates.size()); + + for (CompactionAggregate aggregate : aggregates) + statistics.add(aggregate.getStatistics()); + + return new CompactionStrategyStatistics(metadata, strategy.getClass().getSimpleName(), statistics); + } + + /** + * Return the number of compactions that are still pending; + * @param aggregates the compaction aggregates + * + * @return the number of compactions that are still pending (net yet submitted) + */ + static int numEstimatedCompactions(Collection aggregates) + { + int ret = 0; + for (CompactionAggregate aggregate : aggregates) + ret += aggregate.numEstimatedCompactions(); + + return ret; + } + + /** + * Given a sorted list of compactions, return the first selected pick. + * + * @param aggregates a sorted list of compaction aggregates from most interesting to least interesting, some may be empty + * + * @return the compaction pick of the first aggregate + */ + static CompactionPick getSelected(List aggregates) + { + return aggregates.isEmpty() ? CompactionPick.EMPTY : aggregates.get(0).getSelected(); + } + + /** + * Given a list of sstables, return their average size on disk. + * + * @param sstables the sstables + * @return average sstable size on disk or zero. + */ + static long getAvgSizeBytes(Iterable sstables) + { + long ret = 0; + long num = 0; + for (SSTableReader sstable : sstables) + { + ret += sstable.onDiskLength(); + num++; + } + + return num > 0 ? ret / num : 0; + } + + /** + * Given a list of sstables, return their total size on disk. + * + * @param sstables the sstables + * @return total sstable size on disk or zero. + */ + static long getTotSizeBytes(Iterable sstables) + { + long ret = 0; + for (SSTableReader sstable : sstables) + ret += sstable.onDiskLength(); + + return ret; + } + + /** + * Given a list of sstables, return their total read hotness. + * + * @param sstables the sstables + * @return total read hotness or zero. + */ + static double getTotHotness(Iterable sstables) + { + double ret = 0; + for (SSTableReader sstable : sstables) + ret += sstable.hotness(); + + return ret; + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionAggregateStatistics.java b/src/java/org/apache/cassandra/db/compaction/CompactionAggregateStatistics.java new file mode 100644 index 000000000000..95acf6e0cdc8 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionAggregateStatistics.java @@ -0,0 +1,257 @@ +/* + * 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.compaction; + +import java.io.Serializable; +import java.util.Collection; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableList; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemoryPerSecond; + +/** + * The statistics for a {@link CompactionAggregate}. + *

+ * It must be serializable for JMX and convertible to JSON for insights. The JSON + * properties are published to insights so changing them has a downstream impact. + */ +public class CompactionAggregateStatistics implements Serializable +{ + public static final String NO_SHARD = ""; + + protected static final Collection HEADER = ImmutableList.of("Tot. SSTables", + "Tot. size (bytes)", + "Compactions", + "Comp. SSTables", + "Read (bytes/sec)", + "Write (bytes/sec)", + "Tot. comp. size/Read/Written (bytes)"); + /** The number of compactions that are either pending or in progress */ + protected final int numCompactions; + + /** The number of compactions that are in progress */ + protected final int numCompactionsInProgress; + + /** The total number of sstables, whether they need compacting or not */ + protected final int numSSTables; + + /** The total number of expired sstables */ + protected final int numExpiredSSTables; + + /** The number of sstables that are compaction candidates */ + protected final int numCandidateSSTables; + + /** The number of sstables that are currently compacting */ + protected final int numCompactingSSTables; + + /** The size in bytes (on disk) of the total sstables */ + protected final long sizeInBytes; + + /** The total uncompressed size of the sstables selected for compaction */ + protected final long totBytesToCompact; + + /** The total uncompressed size of the expired sstables that are going to be dropped during compaction */ + protected final long totalBytesToDrop; + + /** The number of bytes read so far for the compactions here - read throughput is calculated based on this */ + protected final long readBytes; + + /** The number of bytes written so far for the compaction here - write throughput is calculated based on this */ + protected final long writtenBytes; + + /** The read throughput in bytes per second */ + protected final double readThroughput; + + /** The write throughput in bytes per second */ + protected final double writeThroughput; + + /** The hotness of this aggregate (where applicable) */ + protected final double hotness; + + CompactionAggregateStatistics(int numCompactions, + int numCompactionsInProgress, + int numSSTables, + int numExpiredSSTables, + int numCandidateSSTables, + int numCompactingSSTables, + long sizeInBytes, + long totBytesToCompact, + long totBytesToDrop, + long readBytes, + long writtenBytes, + long durationNanos, + double hotness) + { + this.numCompactions = numCompactions; + this.numCompactionsInProgress = numCompactionsInProgress; + this.numCandidateSSTables = numCandidateSSTables; + this.numCompactingSSTables = numCompactingSSTables; + this.numSSTables = numSSTables; + this.numExpiredSSTables = numExpiredSSTables; + this.sizeInBytes = sizeInBytes; + this.totBytesToCompact = totBytesToCompact; + this.totalBytesToDrop = totBytesToDrop; + this.readBytes = readBytes; + this.writtenBytes = writtenBytes; + this.readThroughput = durationNanos == 0 ? 0 : ((double) readBytes / durationNanos) * TimeUnit.SECONDS.toNanos(1); + this.writeThroughput = durationNanos == 0 ? 0 : ((double) writtenBytes / durationNanos) * TimeUnit.SECONDS.toNanos(1); + this.hotness = hotness; + } + + CompactionAggregateStatistics(CompactionAggregateStatistics base) + { + this.numCompactions = base.numCompactions; + this.numCompactionsInProgress = base.numCompactionsInProgress; + this.numCandidateSSTables = base.numCandidateSSTables; + this.numCompactingSSTables = base.numCompactingSSTables; + this.numExpiredSSTables = base.numExpiredSSTables; + this.numSSTables = base.numSSTables; + this.sizeInBytes = base.sizeInBytes; + this.totBytesToCompact = base.totBytesToCompact; + this.totalBytesToDrop = base.totalBytesToDrop; + this.readBytes = base.readBytes; + this.writtenBytes = base.writtenBytes; + this.readThroughput = base.readThroughput; + this.writeThroughput = base.writeThroughput; + this.hotness = base.hotness; + } + + /** The number of compactions that are either pending or in progress */ + @JsonProperty + public int numCompactions() + { + return numCompactions; + } + + /** The number of compactions that are in progress */ + @JsonProperty + public int numCompactionsInProgress() + { + return numCompactionsInProgress; + } + + /** The total number of sstables, whether they need compacting or not */ + @JsonProperty + public int numSSTables() + { + return numSSTables; + } + + /** The number of sstables that are part of this level */ + @JsonProperty + public int numCandidateSSTables() + { + return numCandidateSSTables; + } + + /** The number of sstables that are currently part of a compaction operation */ + @JsonProperty + public int numCompactingSSTables() + { + return numCompactingSSTables; + } + + /** The size in bytes (on disk) of the total sstables */ + public long sizeInBytes() + { + return sizeInBytes; + } + + /** The read throughput in bytes per second */ + @JsonProperty + public double readThroughput() + { + return readThroughput; + } + + /** The write throughput in bytes per second */ + @JsonProperty + public double writeThroughput() + { + return writeThroughput; + } + + /** The total uncompressed size of the sstables selected for compaction */ + @JsonProperty + public long tot() + { + return totBytesToCompact; + } + + /** The number of bytes read so far for the compactions here - read throughput is calculated based on this */ + @JsonProperty + public long read() + { + return readBytes; + } + + /** The number of bytes written so far for the compaction here - write throughput is calculated based on this */ + @JsonProperty + public long written() + { + return writtenBytes; + } + + /** The hotness of this aggregate (where applicable) */ + @JsonProperty + public double hotness() + { + return hotness; + } + + /** The name of the shard, empty if the compaction is not sharded (the default). */ + @JsonProperty + public String shard() + { + return NO_SHARD; + } + + @Override + public String toString() + { + return data().toString(); + } + + protected Collection header() + { + return HEADER; + } + + protected Collection data() + { + return ImmutableList.of(Integer.toString(numSSTables), + prettyPrintMemory(sizeInBytes), + Integer.toString(numCompactions()) + '/' + numCompactionsInProgress(), + Integer.toString(numCandidateSSTables()) + '/' + numCompactingSSTables(), + prettyPrintMemoryPerSecond((long) readThroughput()), + prettyPrintMemoryPerSecond((long) writeThroughput()), + prettyPrintMemory(totBytesToCompact) + '/' + prettyPrintMemory(readBytes) + '/' + prettyPrintMemory(writtenBytes)); + } + + protected String toString(long value) + { + return FBUtilities.prettyPrintMemory(value); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java index e1b0f3258359..2c3b4a9972a9 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java @@ -28,6 +28,7 @@ import org.apache.cassandra.config.Config; import org.apache.cassandra.db.*; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.partitions.Partition; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -67,8 +68,7 @@ protected CompactionController(ColumnFamilyStore cfs, int maxValue) public CompactionController(ColumnFamilyStore cfs, Set compacting, int gcBefore) { - this(cfs, compacting, gcBefore, null, - cfs.getCompactionStrategyManager().getCompactionParams().tombstoneOption()); + this(cfs, compacting, gcBefore, null, cfs.getCompactionParams().tombstoneOption()); } public CompactionController(ColumnFamilyStore cfs, Set compacting, int gcBefore, RateLimiter limiter, TombstoneOption tombstoneOption) @@ -164,7 +164,7 @@ public static Set getFullyExpiredSSTables(ColumnFamilyStore cfSto if (NEVER_PURGE_TOMBSTONES || compacting == null || cfStore.getNeverPurgeTombstones()) return Collections.emptySet(); - if (cfStore.getCompactionStrategyManager().onlyPurgeRepairedTombstones() && !Iterables.all(compacting, SSTableReader::isRepaired)) + if (cfStore.onlyPurgeRepairedTombstones() && !Iterables.all(compacting, SSTableReader::isRepaired)) return Collections.emptySet(); if (ignoreOverlaps) @@ -172,11 +172,11 @@ public static Set getFullyExpiredSSTables(ColumnFamilyStore cfSto Set fullyExpired = new HashSet<>(); for (SSTableReader candidate : compacting) { - if (candidate.getSSTableMetadata().maxLocalDeletionTime < gcBefore) + if (candidate.getMaxLocalDeletionTime() < gcBefore) { fullyExpired.add(candidate); logger.trace("Dropping overlap ignored expired SSTable {} (maxLocalDeletionTime={}, gcBefore={})", - candidate, candidate.getSSTableMetadata().maxLocalDeletionTime, gcBefore); + candidate, candidate.getMaxLocalDeletionTime(), gcBefore); } } return fullyExpired; @@ -189,13 +189,13 @@ public static Set getFullyExpiredSSTables(ColumnFamilyStore cfSto { // Overlapping might include fully expired sstables. What we care about here is // the min timestamp of the overlapping sstables that actually contain live data. - if (sstable.getSSTableMetadata().maxLocalDeletionTime >= gcBefore) + if (sstable.getMaxLocalDeletionTime() >= gcBefore) minTimestamp = Math.min(minTimestamp, sstable.getMinTimestamp()); } for (SSTableReader candidate : compacting) { - if (candidate.getSSTableMetadata().maxLocalDeletionTime < gcBefore) + if (candidate.getMaxLocalDeletionTime() < gcBefore) candidates.add(candidate); else minTimestamp = Math.min(minTimestamp, candidate.getMinTimestamp()); @@ -220,7 +220,7 @@ public static Set getFullyExpiredSSTables(ColumnFamilyStore cfSto else { logger.trace("Dropping expired SSTable {} (maxLocalDeletionTime={}, gcBefore={})", - candidate, candidate.getSSTableMetadata().maxLocalDeletionTime, gcBefore); + candidate, candidate.getMaxLocalDeletionTime(), gcBefore); } } return new HashSet<>(candidates); @@ -257,7 +257,7 @@ public LongPredicate getPurgeEvaluator(DecoratedKey key) { // if we don't have bloom filter(bf_fp_chance=1.0 or filter file is missing), // we check index file instead. - if (sstable.getBloomFilter() instanceof AlwaysPresentFilter && sstable.getPosition(key, SSTableReader.Operator.EQ, false) != null + if (sstable.getBloomFilter() instanceof AlwaysPresentFilter && sstable.checkEntryExists(key, SSTableReader.Operator.EQ, false) || sstable.getBloomFilter().isPresent(key)) { minTimestampSeen = Math.min(minTimestampSeen, sstable.getMinTimestamp()); @@ -295,7 +295,7 @@ public void close() public boolean compactingRepaired() { - return !cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones() || compactingRepaired; + return !cfs.onlyPurgeRepairedTombstones() || compactingRepaired; } boolean provideTombstoneSources() @@ -321,11 +321,7 @@ private UnfilteredRowIterator getShadowIterator(SSTableReader reader, DecoratedK reader.getMaxTimestamp() <= minTimestamp || tombstoneOnly && !reader.mayHaveTombstones()) return null; - RowIndexEntry position = reader.getPosition(key, SSTableReader.Operator.EQ); - if (position == null) - return null; - FileDataInput dfile = openDataFiles.computeIfAbsent(reader, this::openDataFile); - return reader.simpleIterator(dfile, key, position, tombstoneOnly); + return reader.simpleIterator(openDataFiles.computeIfAbsent(reader, this::openDataFile), key, tombstoneOnly); } /** diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java b/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java deleted file mode 100644 index bdddaab61198..000000000000 --- a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java +++ /dev/null @@ -1,214 +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.db.compaction; - -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.function.Predicate; - -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableSet; - -import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.schema.TableMetadata; - -public final class CompactionInfo -{ - public static final String ID = "id"; - public static final String KEYSPACE = "keyspace"; - public static final String COLUMNFAMILY = "columnfamily"; - public static final String COMPLETED = "completed"; - public static final String TOTAL = "total"; - public static final String TASK_TYPE = "taskType"; - public static final String UNIT = "unit"; - public static final String COMPACTION_ID = "compactionId"; - public static final String SSTABLES = "sstables"; - - private final TableMetadata metadata; - private final OperationType tasktype; - private final long completed; - private final long total; - private final Unit unit; - private final UUID compactionId; - private final ImmutableSet sstables; - - public CompactionInfo(TableMetadata metadata, OperationType tasktype, long bytesComplete, long totalBytes, UUID compactionId, Collection sstables) - { - this(metadata, tasktype, bytesComplete, totalBytes, Unit.BYTES, compactionId, sstables); - } - - private CompactionInfo(TableMetadata metadata, OperationType tasktype, long completed, long total, Unit unit, UUID compactionId, Collection sstables) - { - this.tasktype = tasktype; - this.completed = completed; - this.total = total; - this.metadata = metadata; - this.unit = unit; - this.compactionId = compactionId; - this.sstables = ImmutableSet.copyOf(sstables); - } - - /** - * Special compaction info where we always need to cancel the compaction - for example ViewBuilderTask and AutoSavingCache where we don't know - * the sstables at construction - */ - public static CompactionInfo withoutSSTables(TableMetadata metadata, OperationType tasktype, long completed, long total, Unit unit, UUID compactionId) - { - return new CompactionInfo(metadata, tasktype, completed, total, unit, compactionId, ImmutableSet.of()); - } - - /** @return A copy of this CompactionInfo with updated progress. */ - public CompactionInfo forProgress(long complete, long total) - { - return new CompactionInfo(metadata, tasktype, complete, total, unit, compactionId, sstables); - } - - public Optional getKeyspace() - { - return Optional.ofNullable(metadata != null ? metadata.keyspace : null); - } - - public Optional getTable() - { - return Optional.ofNullable(metadata != null ? metadata.name : null); - } - - public TableMetadata getTableMetadata() - { - return metadata; - } - - public long getCompleted() - { - return completed; - } - - public long getTotal() - { - return total; - } - - public OperationType getTaskType() - { - return tasktype; - } - - public UUID getTaskId() - { - return compactionId; - } - - public Unit getUnit() - { - return unit; - } - - public Set getSSTables() - { - return sstables; - } - - @Override - public String toString() - { - if (metadata != null) - { - return String.format("%s(%s, %s / %s %s)@%s(%s, %s)", - tasktype, compactionId, completed, total, unit, - metadata.id, metadata.keyspace, metadata.name); - } - else - { - return String.format("%s(%s, %s / %s %s)", - tasktype, compactionId, completed, total, unit); - } - } - - public Map asMap() - { - Map ret = new HashMap(); - ret.put(ID, metadata != null ? metadata.id.toString() : ""); - ret.put(KEYSPACE, getKeyspace().orElse(null)); - ret.put(COLUMNFAMILY, getTable().orElse(null)); - ret.put(COMPLETED, Long.toString(completed)); - ret.put(TOTAL, Long.toString(total)); - ret.put(TASK_TYPE, tasktype.toString()); - ret.put(UNIT, unit.toString()); - ret.put(COMPACTION_ID, compactionId == null ? "" : compactionId.toString()); - ret.put(SSTABLES, Joiner.on(',').join(sstables)); - return ret; - } - - boolean shouldStop(Predicate sstablePredicate) - { - if (sstables.isEmpty()) - { - return true; - } - return sstables.stream().anyMatch(sstablePredicate); - } - - public static abstract class Holder - { - private volatile boolean stopRequested = false; - public abstract CompactionInfo getCompactionInfo(); - - public void stop() - { - stopRequested = true; - } - - /** - * if this compaction involves several/all tables we can safely check globalCompactionsPaused - * in isStopRequested() below - */ - public abstract boolean isGlobal(); - - public boolean isStopRequested() - { - return stopRequested || (isGlobal() && CompactionManager.instance.isGlobalCompactionPaused()); - } - } - - public enum Unit - { - BYTES("bytes"), RANGES("token range parts"), KEYS("keys"); - - private final String name; - - Unit(String name) - { - this.name = name; - } - - @Override - public String toString() - { - return this.name; - } - - public static boolean isFileSize(String unit) - { - return BYTES.toString().equals(unit); - } - } -} diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionInterruptedException.java b/src/java/org/apache/cassandra/db/compaction/CompactionInterruptedException.java index 129d9fc0ff5c..a28c39526a14 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionInterruptedException.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionInterruptedException.java @@ -21,7 +21,7 @@ public class CompactionInterruptedException extends RuntimeException { private static final long serialVersionUID = -8651427062512310398L; - public CompactionInterruptedException(CompactionInfo info) + public CompactionInterruptedException(AbstractTableOperation.OperationProgress info) { super("Compaction interrupted: " + info); } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java index ec6a4d464c21..b247480bcdce 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Ordering; +import org.apache.cassandra.index.transactions.IndexTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.TableMetadata; @@ -37,6 +38,7 @@ import org.apache.cassandra.index.transactions.CompactionTransaction; import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.schema.CompactionParams.TombstoneOption; +import org.apache.cassandra.utils.Throwables; /** * Merge multiple iterators over the content of sstable into a "compacted" iterator. @@ -54,7 +56,7 @@ *

  • keep tracks of the compaction progress.
  • * */ -public class CompactionIterator extends CompactionInfo.Holder implements UnfilteredPartitionIterator +public class CompactionIterator implements UnfilteredPartitionIterator { private static final long UNFILTERED_TO_UPDATE_PROGRESS = 100; @@ -66,44 +68,40 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte private final UUID compactionId; private final long totalBytes; - private long bytesRead; - private long totalSourceCQLRows; + private volatile long[] bytesReadByLevel; - /* - * counters for merged rows. - * array index represents (number of merged rows - 1), so index 0 is counter for no merge (1 row), - * index 1 is counter for 2 rows merged, and so on. + /** + * Merged frequency counters for partitions and rows (AKA histograms). + * The array index represents the number of sstables containing the row or partition minus one. So index 0 contains + * the number of rows or partitions coming from a single sstable (therefore copied rather than merged), index 1 contains + * the number of rows or partitions coming from two sstables and so forth. */ - private final long[] mergeCounters; + private final long[] mergedPartitionsHistogram; + private final long[] mergedRowsHistogram; private final UnfilteredPartitionIterator compacted; - private final ActiveCompactionsTracker activeCompactions; - - public CompactionIterator(OperationType type, List scanners, AbstractCompactionController controller, int nowInSec, UUID compactionId) - { - this(type, scanners, controller, nowInSec, compactionId, ActiveCompactionsTracker.NOOP); - } + private final TableOperation op; @SuppressWarnings("resource") // We make sure to close mergedIterator in close() and CompactionIterator is itself an AutoCloseable - public CompactionIterator(OperationType type, List scanners, AbstractCompactionController controller, int nowInSec, UUID compactionId, ActiveCompactionsTracker activeCompactions) + public CompactionIterator(OperationType type, List scanners, AbstractCompactionController controller, int nowInSec, UUID compactionId) { this.controller = controller; this.type = type; this.scanners = scanners; this.nowInSec = nowInSec; this.compactionId = compactionId; - this.bytesRead = 0; + this.bytesReadByLevel = new long[LeveledGenerations.MAX_LEVEL_COUNT]; long bytes = 0; for (ISSTableScanner scanner : scanners) bytes += scanner.getLengthInBytes(); this.totalBytes = bytes; - this.mergeCounters = new long[scanners.size()]; + this.mergedPartitionsHistogram = new long[scanners.size()]; + this.mergedRowsHistogram = new long[scanners.size()]; // note that we leak `this` from the constructor when calling beginCompaction below, this means we have to get the sstables before // calling that to avoid a NPE. sstables = scanners.stream().map(ISSTableScanner::getBackingSSTables).flatMap(Collection::stream).collect(ImmutableSet.toImmutableSet()); - this.activeCompactions = activeCompactions == null ? ActiveCompactionsTracker.NOOP : activeCompactions; - this.activeCompactions.beginCompaction(this); // note that CompactionTask also calls this, but CT only creates CompactionIterator with a NOOP ActiveCompactions + op = createOperation(); UnfilteredPartitionIterator merged = scanners.isEmpty() ? EmptyIterators.unfilteredPartition(controller.cfs.metadata()) @@ -111,7 +109,36 @@ public CompactionIterator(OperationType type, List scanners, Ab merged = Transformation.apply(merged, new GarbageSkipper(controller)); merged = Transformation.apply(merged, new Purger(controller, nowInSec)); merged = DuplicateRowChecker.duringCompaction(merged, type); - compacted = Transformation.apply(merged, new AbortableUnfilteredPartitionTransformation(this)); + compacted = Transformation.apply(merged, new AbortableUnfilteredPartitionTransformation(op)); + } + + protected TableOperation createOperation() + { + return new AbstractTableOperation() { + + @Override + public OperationProgress getProgress() + { + return new AbstractTableOperation.OperationProgress(controller.cfs.metadata(), type, bytesRead(), totalBytes, compactionId, sstables); + } + + @Override + public boolean isGlobal() + { + return false; + } + }; + } + + /** + * @return A {@link TableOperation} backed by this iterator. This operation can be observed for progress + * and for interrupting provided that it is registered with a {@link TableOperationObserver}, normally the + * metrics in the compaction manager. The caller is responsible for registering the operation and checking + * {@link TableOperation#isStopRequested()}. + */ + public TableOperation getOperation() + { + return op; } public TableMetadata metadata() @@ -119,35 +146,45 @@ public TableMetadata metadata() return controller.cfs.metadata(); } - public CompactionInfo getCompactionInfo() + long bytesRead() { - return new CompactionInfo(controller.cfs.metadata(), - type, - bytesRead, - totalBytes, - compactionId, - sstables); + long[] bytesReadByLevel = this.bytesReadByLevel; + return Arrays.stream(bytesReadByLevel).reduce(Long::sum).orElse(0L); } - public boolean isGlobal() + long bytesRead(int level) { - return false; + return level >= 0 && level < bytesReadByLevel.length ? bytesReadByLevel[level] : 0; + } + + long totalBytes() + { + return totalBytes; + } + + long totalSourcePartitions() + { + return Arrays.stream(mergedPartitionsHistogram).reduce(0L, Long::sum); + } + + long totalSourceRows() + { + return Arrays.stream(mergedRowsHistogram).reduce(0L, Long::sum); } - private void updateCounterFor(int rows) + long[] mergedPartitionsHistogram() { - assert rows > 0 && rows - 1 < mergeCounters.length; - mergeCounters[rows - 1] += 1; + return mergedPartitionsHistogram; } - public long[] getMergedRowCounts() + long[] mergedRowsHistogram() { - return mergeCounters; + return mergedRowsHistogram; } - public long getTotalSourceCQLRows() + public boolean isGlobal() { - return totalSourceCQLRows; + return false; } private UnfilteredPartitionIterators.MergeListener listener() @@ -156,51 +193,18 @@ private UnfilteredPartitionIterators.MergeListener listener() { public UnfilteredRowIterators.MergeListener getRowMergeListener(DecoratedKey partitionKey, List versions) { - int merged = 0; + int numVersions = 0; for (int i=0, isize=versions.size(); i 0; - - CompactionIterator.this.updateCounterFor(merged); + mergedPartitionsHistogram[numVersions - 1] += 1; - if (type != OperationType.COMPACTION || !controller.cfs.indexManager.hasIndexes()) - return null; - - Columns statics = Columns.NONE; - Columns regulars = Columns.NONE; - for (int i=0, isize=versions.size(); i 0 && numVersions - 1 < mergedRowsHistogram.length; + mergedRowsHistogram[numVersions - 1] += 1; + + if (indexTransaction != null) + { + indexTransaction.start(); + indexTransaction.onRowMerge(merged, versions); + indexTransaction.commit(); + } + return merged; } @@ -232,12 +250,49 @@ public void close() }; } + private CompactionTransaction getIndexTransaction(DecoratedKey partitionKey, List versions) + { + if (type != OperationType.COMPACTION || !controller.cfs.indexManager.handles(IndexTransaction.Type.COMPACTION)) + return null; + + Columns statics = Columns.NONE; + Columns regulars = Columns.NONE; + for (int i=0, isize=versions.size(); i= 0 && level < bytesReadByLevel.length) + bytesReadByLevel[level] += n; + } + this.bytesReadByLevel = bytesReadByLevel; } public boolean hasNext() @@ -257,19 +312,14 @@ public void remove() public void close() { - try - { - compacted.close(); - } - finally - { - activeCompactions.finishCompaction(this); - } + updateBytesRead(); + + Throwables.maybeFail(Throwables.close(null, compacted)); } public String toString() { - return this.getCompactionInfo().toString(); + return String.format("%s: %s, (%d/%d)", type, metadata(), bytesRead(), totalBytes()); } private class Purger extends PurgeFunction @@ -284,7 +334,7 @@ private class Purger extends PurgeFunction private Purger(AbstractCompactionController controller, int nowInSec) { super(nowInSec, controller.gcBefore, controller.compactingRepaired() ? Integer.MAX_VALUE : Integer.MIN_VALUE, - controller.cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones(), + controller.cfs.onlyPurgeRepairedTombstones(), controller.cfs.metadata.get().enforceStrictLiveness()); this.controller = controller; } @@ -306,7 +356,6 @@ protected void onNewPartition(DecoratedKey key) @Override protected void updateProgress() { - totalSourceCQLRows++; if ((++compactedUnfiltered) % UNFILTERED_TO_UPDATE_PROGRESS == 0) updateBytesRead(); } @@ -558,34 +607,36 @@ protected UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition private static class AbortableUnfilteredPartitionTransformation extends Transformation { private final AbortableUnfilteredRowTransformation abortableIter; + private final TableOperation op; - private AbortableUnfilteredPartitionTransformation(CompactionIterator iter) + private AbortableUnfilteredPartitionTransformation(TableOperation op) { - this.abortableIter = new AbortableUnfilteredRowTransformation(iter); + this.op = op; + this.abortableIter = new AbortableUnfilteredRowTransformation(op); } @Override protected UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition) { - if (abortableIter.iter.isStopRequested()) - throw new CompactionInterruptedException(abortableIter.iter.getCompactionInfo()); + if (op.isStopRequested()) + throw new CompactionInterruptedException(op.getProgress()); return Transformation.apply(partition, abortableIter); } } private static class AbortableUnfilteredRowTransformation extends Transformation { - private final CompactionIterator iter; + private final TableOperation op; - private AbortableUnfilteredRowTransformation(CompactionIterator iter) + private AbortableUnfilteredRowTransformation(TableOperation op) { - this.iter = iter; + this.op = op; } public Row applyToRow(Row row) { - if (iter.isStopRequested()) - throw new CompactionInterruptedException(iter.getCompactionInfo()); + if (op.isStopRequested()) + throw new CompactionInterruptedException(op.getProgress()); return row; } } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java index f473be705918..310e6148842d 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionLogger.java @@ -18,21 +18,30 @@ package org.apache.cassandra.db.compaction; +import java.io.Closeable; import java.io.IOException; import java.io.OutputStreamWriter; -import java.lang.ref.WeakReference; -import java.nio.file.*; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; import java.util.Collection; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.MapMaker; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,31 +50,22 @@ import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.NoSpamLogger; +import org.apache.cassandra.utils.Throwables; +/** + * This is a Compaction logger that logs compaction events in a file called compactions.log. + * It was added by CASSANDRA-10805. + */ public class CompactionLogger { - public interface Strategy - { - JsonNode sstable(SSTableReader sstable); - - JsonNode options(); - - static Strategy none = new Strategy() - { - public JsonNode sstable(SSTableReader sstable) - { - return null; - } - - public JsonNode options() - { - return null; - } - }; - } + private static final DateTimeFormatter dateFormatter = DateTimeFormatter + .ofPattern("yyyy-MM-dd' 'HH:mm:ss.SSS") + .withZone(ZoneId.systemDefault() ); /** * This will produce the compaction strategy's starting information. @@ -78,8 +78,13 @@ public interface StrategySummary /** * This is an interface to allow writing to a different interface. */ - public interface Writer + public interface Writer extends Closeable { + /** + * @param toWrite This should be written out to the medium capturing the logs + */ + void write(String toWrite); + /** * This is used when we are already trying to write out the start of a * @param statement This should be written out to the medium capturing the logs @@ -94,141 +99,141 @@ public interface Writer * @param tag This is an identifier for a strategy; each strategy should have a distinct Object */ void write(JsonNode statement, StrategySummary summary, Object tag); - } - private interface CompactionStrategyAndTableFunction - { - JsonNode apply(AbstractCompactionStrategy strategy, SSTableReader sstable); + /** + * Closes the writer + */ + @Override + void close(); } private static final JsonNodeFactory json = JsonNodeFactory.instance; private static final Logger logger = LoggerFactory.getLogger(CompactionLogger.class); - private static final Writer serializer = new CompactionLogSerializer(); - private final WeakReference cfsRef; - private final WeakReference csmRef; + + private static final ExecutorService loggerService = Executors.newFixedThreadPool(1, new NamedThreadFactory("compaction-logger")); + private static final Writer jsonWriter = new CompactionLogSerializer("compaction", "log", loggerService); + + private final String keyspace; + private final String table; private final AtomicInteger identifier = new AtomicInteger(0); - private final Map compactionStrategyMapping = new MapMaker().weakKeys().makeMap(); + private final Map compactionStrategyMapping = new MapMaker().weakKeys().makeMap(); + private final Map> csvWriters = new MapMaker().makeMap(); private final AtomicBoolean enabled = new AtomicBoolean(false); - public CompactionLogger(ColumnFamilyStore cfs, CompactionStrategyManager csm) + CompactionLogger(TableMetadata metadata) { - csmRef = new WeakReference<>(csm); - cfsRef = new WeakReference<>(cfs); + this.keyspace = metadata.keyspace; + this.table = metadata.name; } - private void forEach(Consumer consumer) + void strategyCreated(CompactionStrategy strategy) { - CompactionStrategyManager csm = csmRef.get(); - if (csm == null) - return; - csm.getStrategies() - .forEach(l -> l.forEach(consumer)); + compactionStrategyMapping.computeIfAbsent(strategy, s -> String.valueOf(identifier.getAndIncrement())); + } + + /** + * Visit all the strategies. + * + * @param consumer a consumer function that receives all the strategies one by one + */ + private void visitStrategies(Consumer consumer) + { + compactionStrategyMapping.keySet().forEach(consumer); } - private ArrayNode compactionStrategyMap(Function select) + /** + * Rely on {@link this#visitStrategies(Consumer)} to visit all the strategies + * and add the properties extracted by the function passed in to a json node that is returned. + * + * @param select a function that given a strategy returns a json node + * + * @return a json node containing information on all the strategies returned by the strategy manager and the function passed in. + */ + private ArrayNode getStrategiesJsonNode(Function select) { ArrayNode node = json.arrayNode(); - forEach(acs -> node.add(select.apply(acs))); + visitStrategies(acs -> node.add(select.apply(acs))); return node; } - private ArrayNode sstableMap(Collection sstables, CompactionStrategyAndTableFunction csatf) + private ArrayNode sstableMap(Collection sstables) { - CompactionStrategyManager csm = csmRef.get(); ArrayNode node = json.arrayNode(); - if (csm == null) - return node; - sstables.forEach(t -> node.add(csatf.apply(csm.getCompactionStrategyFor(t), t))); + sstables.forEach(t -> node.add(describeSSTable(t))); return node; } - private String getId(AbstractCompactionStrategy strategy) + private String getId(CompactionStrategy strategy) { - return compactionStrategyMapping.computeIfAbsent(strategy, s -> String.valueOf(identifier.getAndIncrement())); + return compactionStrategyMapping.getOrDefault(strategy, "-1"); // there should always be a strategy because of strategyCreated() } - private JsonNode formatSSTables(AbstractCompactionStrategy strategy) + private JsonNode formatSSTables(CompactionStrategy strategy) { ArrayNode node = json.arrayNode(); - CompactionStrategyManager csm = csmRef.get(); - ColumnFamilyStore cfs = cfsRef.get(); - if (csm == null || cfs == null) - return node; - for (SSTableReader sstable : cfs.getLiveSSTables()) - { - if (csm.getCompactionStrategyFor(sstable) == strategy) - node.add(formatSSTable(strategy, sstable)); - } + for (SSTableReader sstable : strategy.getSSTables()) + node.add(formatSSTable(sstable)); + return node; } - private JsonNode formatSSTable(AbstractCompactionStrategy strategy, SSTableReader sstable) + private JsonNode formatSSTable(SSTableReader sstable) { ObjectNode node = json.objectNode(); node.put("generation", sstable.descriptor.generation); node.put("version", sstable.descriptor.version.getVersion()); node.put("size", sstable.onDiskLength()); - JsonNode logResult = strategy.strategyLogger().sstable(sstable); - if (logResult != null) - node.set("details", logResult); + + // The details are only relevant or available for some strategies, e.g. LCS or Date tiered but + // it doesn't hurt to log them all the time in order to simplify things + ObjectNode details = json.objectNode(); + details.put("level", sstable.getSSTableLevel()); + details.put("min_token", sstable.first.getToken().toString()); + details.put("max_token", sstable.last.getToken().toString()); + details.put("min_timestamp", sstable.getMinTimestamp()); + details.put("max_timestamp", sstable.getMaxTimestamp()); + + node.put("details", details); + return node; } - private JsonNode startStrategy(AbstractCompactionStrategy strategy) + private JsonNode getStrategyDetails(CompactionStrategy strategy) { ObjectNode node = json.objectNode(); - CompactionStrategyManager csm = csmRef.get(); - if (csm == null) - return node; node.put("strategyId", getId(strategy)); node.put("type", strategy.getName()); node.set("tables", formatSSTables(strategy)); - node.put("repaired", csm.isRepaired(strategy)); - List folders = csm.getStrategyFolders(strategy); - ArrayNode folderNode = json.arrayNode(); - for (String folder : folders) - { - folderNode.add(folder); - } - node.set("folders", folderNode); - - JsonNode logResult = strategy.strategyLogger().options(); - if (logResult != null) - node.set("options", logResult); return node; } - private JsonNode shutdownStrategy(AbstractCompactionStrategy strategy) + private JsonNode getStrategyId(CompactionStrategy strategy) { ObjectNode node = json.objectNode(); node.put("strategyId", getId(strategy)); return node; } - private JsonNode describeSSTable(AbstractCompactionStrategy strategy, SSTableReader sstable) + private JsonNode describeSSTable(SSTableReader sstable) { ObjectNode node = json.objectNode(); - node.put("strategyId", getId(strategy)); - node.set("table", formatSSTable(strategy, sstable)); + node.put("table", formatSSTable(sstable)); return node; } - private void describeStrategy(ObjectNode node) + private void maybeAddSchemaAndTimeInfo(ObjectNode node) { - ColumnFamilyStore cfs = cfsRef.get(); - if (cfs == null) - return; - node.put("keyspace", cfs.keyspace.getName()); - node.put("table", cfs.getTableName()); + node.put("keyspace", keyspace); + node.put("table", table); node.put("time", System.currentTimeMillis()); } - private JsonNode startStrategies() + private JsonNode getEventJsonNode() { ObjectNode node = json.objectNode(); node.put("type", "enable"); - describeStrategy(node); - node.set("strategies", compactionStrategyMap(this::startStrategy)); + maybeAddSchemaAndTimeInfo(node); + node.set("strategies", getStrategiesJsonNode(this::getStrategyDetails)); return node; } @@ -236,7 +241,7 @@ public void enable() { if (enabled.compareAndSet(false, true)) { - serializer.writeStart(startStrategies(), this); + jsonWriter.writeStart(getEventJsonNode(), this); } } @@ -246,21 +251,28 @@ public void disable() { ObjectNode node = json.objectNode(); node.put("type", "disable"); - describeStrategy(node); - node.set("strategies", compactionStrategyMap(this::shutdownStrategy)); - serializer.write(node, this::startStrategies, this); + maybeAddSchemaAndTimeInfo(node); + node.set("strategies", getStrategiesJsonNode(this::getStrategyId)); + jsonWriter.write(node, this::getEventJsonNode, this); + + visitStrategies(strategy -> csvWriters.computeIfPresent(strategy, (s, writers) -> { writers.values().forEach(Writer::close); return null; })); } } + public boolean enabled() + { + return enabled.get(); + } + public void flush(Collection sstables) { if (enabled.get()) { ObjectNode node = json.objectNode(); node.put("type", "flush"); - describeStrategy(node); - node.set("tables", sstableMap(sstables, this::describeSSTable)); - serializer.write(node, this::startStrategies, this); + maybeAddSchemaAndTimeInfo(node); + node.set("tables", sstableMap(sstables)); + jsonWriter.write(node, this::getEventJsonNode, this); } } @@ -270,46 +282,119 @@ public void compaction(long startTime, Collection input, long end { ObjectNode node = json.objectNode(); node.put("type", "compaction"); - describeStrategy(node); + maybeAddSchemaAndTimeInfo(node); node.put("start", String.valueOf(startTime)); node.put("end", String.valueOf(endTime)); - node.set("input", sstableMap(input, this::describeSSTable)); - node.set("output", sstableMap(output, this::describeSSTable)); - serializer.write(node, this::startStrategies, this); + node.set("input", sstableMap(input)); + node.set("output", sstableMap(output)); + jsonWriter.write(node, this::getEventJsonNode, this); } } - public void pending(AbstractCompactionStrategy strategy, int remaining) + public void pending(CompactionStrategy strategy, int remaining) { if (remaining != 0 && enabled.get()) { ObjectNode node = json.objectNode(); node.put("type", "pending"); - describeStrategy(node); + maybeAddSchemaAndTimeInfo(node); node.put("strategyId", getId(strategy)); node.put("pending", remaining); - serializer.write(node, this::startStrategies, this); + jsonWriter.write(node, this::getEventJsonNode, this); } } + /** + * Write the strategy statistics formatted as CSV. + **/ + public void statistics(CompactionStrategy strategy, String event, CompactionStrategyStatistics statistics) + { + if (logger.isTraceEnabled()) + logger.trace("Compaction statistics for strategy {} and event {}: {}", strategy, event, statistics); + + if (!enabled.get()) + return; + + for (CompactionAggregateStatistics aggregateStatistics : statistics.aggregates()) + { + Writer writer = getCsvWriter(strategy, statistics.getHeader(), aggregateStatistics); + writer.write(String.join(",", Iterables.concat(ImmutableList.of(currentTime(), event), aggregateStatistics.data())) + System.lineSeparator()); + } + } + + private Writer getCsvWriter(CompactionStrategy strategy, Collection header, CompactionAggregateStatistics statistics) + { + Map writers = csvWriters.get(strategy); + if (writers == null) + { + writers = new MapMaker().makeMap(); + if (csvWriters.putIfAbsent(strategy, writers) != null) + { + writers = csvWriters.get(strategy); + } + } + + String shard = statistics.shard(); + Writer writer = writers.get(shard); + if (writer != null) + return writer; + + String fileName = String.format("compaction-%s-%s-%s-%s", + strategy.getName(), + keyspace, + table, + getId(strategy)); + + if (!shard.isEmpty()) + fileName += '-' + shard; + + writer = new CompactionLogSerializer(fileName, "csv", loggerService); + if (writers.putIfAbsent(shard, writer) == null) + { + writer.write(String.join(",", Iterables.concat(ImmutableList.of("Timestamp", "Event"), header)) + System.lineSeparator()); + return writer; + } + else + { + writer.close(); + return writers.get(shard); + } + } + + private String currentTime() + { + return dateFormatter.format(Instant.ofEpochMilli(System.currentTimeMillis())); + } + private static class CompactionLogSerializer implements Writer { private static final String logDirectory = System.getProperty("cassandra.logdir", "."); - private final ExecutorService loggerService = Executors.newFixedThreadPool(1); + // This is only accessed on the logger service thread, so it does not need to be thread safe - private final Set rolled = new HashSet<>(); + private final String fileName; + private final String fileExt; + private final ExecutorService loggerService; + private final Set rolled; private OutputStreamWriter stream; - private static OutputStreamWriter createStream() throws IOException + CompactionLogSerializer(String fileName, String fileExt, ExecutorService loggerService) + { + this.fileName = fileName; + this.fileExt = fileExt; + this.loggerService = loggerService; + this.rolled = new HashSet<>(); + } + + private OutputStreamWriter createStream() throws IOException { int count = 0; - Path compactionLog = Paths.get(logDirectory, "compaction.log"); + Path compactionLog = Paths.get(logDirectory, String.format("%s.%s", fileName, fileExt)); if (Files.exists(compactionLog)) { Path tryPath = compactionLog; while (Files.exists(tryPath)) { - tryPath = Paths.get(logDirectory, String.format("compaction-%d.log", count++)); + tryPath = Paths.get(logDirectory, String.format("%s-%d.%s", fileName, count++, fileExt)); } Files.move(compactionLog, tryPath); } @@ -317,44 +402,71 @@ private static OutputStreamWriter createStream() throws IOException return new OutputStreamWriter(Files.newOutputStream(compactionLog, StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)); } - private void writeLocal(String toWrite) + private interface ThrowingConsumer { - try - { - if (stream == null) - stream = createStream(); - stream.write(toWrite); - stream.flush(); - } - catch (IOException ioe) + void accept(T stream) throws IOException; + } + + private void performWrite(ThrowingConsumer writeTask) + { + loggerService.execute(() -> { - // We'll drop the change and log the error to the logger. - NoSpamLogger.log(logger, NoSpamLogger.Level.ERROR, 1, TimeUnit.MINUTES, - "Could not write to the log file: {}", ioe); - } + try + { + if (stream == null) + stream = createStream(); + + writeTask.accept(stream); + stream.flush(); + } + catch (IOException ioe) + { + // We'll drop the change and log the error to the logger. + NoSpamLogger.log(logger, NoSpamLogger.Level.ERROR, 1, TimeUnit.MINUTES, + "Could not write to the log file: {}", ioe); + } + }); + } + public void write(String toWrite) + { + performWrite(s -> s.write(toWrite)); } public void writeStart(JsonNode statement, Object tag) { final String toWrite = statement.toString() + System.lineSeparator(); - loggerService.execute(() -> { + performWrite(s -> { rolled.add(tag); - writeLocal(toWrite); + s.write(toWrite); }); } public void write(JsonNode statement, StrategySummary summary, Object tag) { final String toWrite = statement.toString() + System.lineSeparator(); - loggerService.execute(() -> { + performWrite(s -> { if (!rolled.contains(tag)) { - writeLocal(summary.getSummary().toString() + System.lineSeparator()); + s.write(toWrite); rolled.add(tag); } - writeLocal(toWrite); }); } + + public void close() + { + if (stream != null) + { + Throwable err = Throwables.close(null, stream); + if (err != null) + { + JVMStabilityInspector.inspectThrowable(err); + logger.error("Failed to close {}: {}", String.format("%s.%s", fileName, fileExt), err); + } + + stream = null; + } + } } } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index 8a0926d48c47..e76eb2739497 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -17,7 +17,9 @@ */ package org.apache.cassandra.db.compaction; +import java.io.Closeable; import java.io.File; +import java.io.IOError; import java.io.IOException; import java.util.*; import java.util.concurrent.*; @@ -30,10 +32,18 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.base.Predicates; import com.google.common.collect.*; +import com.google.common.primitives.Longs; import com.google.common.util.concurrent.*; +import org.apache.cassandra.config.Config; import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.io.FSDiskFullWriteError; +import org.apache.cassandra.io.FSError; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.ScannerList; import org.apache.cassandra.locator.RangesAtEndpoint; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +58,6 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.db.*; -import org.apache.cassandra.db.compaction.CompactionInfo.Holder; import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.lifecycle.SSTableIntervalTree; @@ -115,6 +124,8 @@ protected Boolean initialValue() return false; } }; + private static final int ACQUIRE_GRANULARITY = + Integer.getInteger(Config.PROPERTY_PREFIX + "compaction_rate_limit_granularity_in_kb", 128) * 1024; static { @@ -132,7 +143,7 @@ protected Boolean initialValue() @VisibleForTesting final Multiset compactingCF = ConcurrentHashMultiset.create(); - public final ActiveCompactions active = new ActiveCompactions(); + public final ActiveOperations active = new ActiveOperations(); // used to temporarily pause non-strategy managed compactions (like index summary redistribution) private final AtomicInteger globalCompactionPauseCount = new AtomicInteger(0); @@ -172,16 +183,21 @@ public void setRate(final double throughPutMbPerSec) } /** - * Call this whenever a compaction might be needed on the given columnfamily. + * Call this whenever a compaction might be needed on the given column family store. * It's okay to over-call (within reason) if a call is unnecessary, it will * turn into a no-op in the bucketing/candidate-scan phase. */ - public List> submitBackground(final ColumnFamilyStore cfs) + public Future submitBackground(final ColumnFamilyStore cfs) { + if (!cfs.isValid()) + { + logger.trace("Aborting compaction for dropped CF {}.{}", cfs.keyspace.getName(), cfs.name); + return CompletableFuture.completedFuture(null); + } if (cfs.isAutoCompactionDisabled()) { logger.trace("Autocompaction is disabled"); - return Collections.emptyList(); + return CompletableFuture.completedFuture(null); } /** @@ -189,27 +205,53 @@ public List> submitBackground(final ColumnFamilyStore cfs) * we can wait for the current compaction to finish and re-submit when more information is available. * Otherwise, we should submit at least one task to prevent starvation by busier CFs, and more if there * are idle threads stil. (CASSANDRA-4310) + * + ** We will check again when the scheduled task is executed. */ + if (hasEnoughCompactionsRunning(cfs)) + return CompletableFuture.completedFuture(null); + + logger.trace("Scheduling a background task check for {}.{} with {}", + cfs.keyspace.getName(), + cfs.name, + cfs.getCompactionStrategy().getName()); + + ListenableFuture fut = executor.submitIfRunning(new BackgroundCompactionCandidate(cfs), "background task"); + if (fut.isCancelled()) + compactingCF.remove(cfs); + return fut; + } + + private boolean hasEnoughCompactionsRunning(ColumnFamilyStore cfs) + { int count = compactingCF.count(cfs); if (count > 0 && executor.getActiveCount() >= executor.getMaximumPoolSize()) { logger.trace("Background compaction is still running for {}.{} ({} remaining). Skipping", cfs.keyspace.getName(), cfs.name, count); - return Collections.emptyList(); + return true; } + return false; + } - logger.trace("Scheduling a background task check for {}.{} with {}", - cfs.keyspace.getName(), - cfs.name, - cfs.getCompactionStrategyManager().getName()); - - List> futures = new ArrayList<>(1); - Future fut = executor.submitIfRunning(new BackgroundCompactionCandidate(cfs), "background task"); - if (!fut.isCancelled()) - futures.add(fut); + private static void handleCompactionError(Throwable t, ColumnFamilyStore cfs) + { + t = Throwables.unwrapped(t); + // FSDiskFullWriteErrors caught during compaction are expected to be recoverable, so we don't explicitly + // trigger the disk failure policy because of them (see CASSANDRA-12385). + if (t instanceof IOError && !(t instanceof FSDiskFullWriteError)) + { + logger.error("Potentially unrecoverable error during background compaction of table {}", cfs, t); + // Strictly speaking it's also possible to hit a read-related IOError during compaction, although the + // chances for that are much lower than the chances for write-related IOError. If we want to handle that, + // we might have to rely on error message parsing... + t = t instanceof FSError ? t : new FSWriteError(t); + JVMStabilityInspector.inspectThrowable(t); + } else - compactingCF.remove(cfs); - return futures; + { + logger.error("Exception during background compaction of table {}", cfs, t); + } } public boolean isCompacting(Iterable cfses, Predicate sstablePredicate) @@ -233,9 +275,9 @@ public void forceShutdown() cacheCleanupExecutor.shutdown(); // interrupt compactions and validations - for (Holder compactionHolder : active.getCompactions()) + for (TableOperation operationSource : active.getTableOperations()) { - compactionHolder.stop(); + operationSource.stop(); } // wait for tasks to terminate @@ -276,58 +318,129 @@ class BackgroundCompactionCandidate implements Runnable public void run() { - boolean ranCompaction = false; - try + logger.trace("Checking {}.{}", cfs.keyspace.getName(), cfs.name); + if (!cfs.isValid()) + { + compactingCF.remove(cfs); + logger.trace("Aborting compaction for dropped CF"); + return; + } + + Collection tasks; + synchronized (CompactionManager.instance) { - logger.trace("Checking {}.{}", cfs.keyspace.getName(), cfs.name); - if (!cfs.isValid()) + if (hasEnoughCompactionsRunning(cfs)) { - logger.trace("Aborting compaction for dropped CF"); + compactingCF.remove(cfs); return; } - CompactionStrategyManager strategy = cfs.getCompactionStrategyManager(); - AbstractCompactionTask task = strategy.getNextBackgroundTask(getDefaultGcBefore(cfs, FBUtilities.nowInSeconds())); - if (task == null) + tasks = cfs.getCompactionStrategy() + .getNextBackgroundTasks(getDefaultGcBefore(cfs, FBUtilities.nowInSeconds())); + } + + CompletableFuture fut = null; + if (tasks.isEmpty()) + { + if (DatabaseDescriptor.automaticSSTableUpgrade()) + fut = maybeRunUpgradeTask(); + } + else + { + CompletableFuture[] futures = new CompletableFuture[tasks.size()]; + int i = 0; + for (AbstractCompactionTask task : tasks) + { + futures[i++] = CompletableFuture.runAsync(() -> task.execute(active), executor); + } + fut = CompletableFuture.allOf(futures); + } + + if (fut != null) + { + try { - if (DatabaseDescriptor.automaticSSTableUpgrade()) - ranCompaction = maybeRunUpgradeTask(strategy); + fut.get(); } - else + catch (InterruptedException | ExecutionException e) { - task.execute(active); - ranCompaction = true; + handleCompactionError(e, cfs); + logger.warn("Aborting compaction due to ", e); + if (e instanceof InterruptedException) + Thread.currentThread().interrupt(); } + finally + { + compactingCF.remove(cfs); + } + // Since we have ran at least one task and thus the set of sstables has changed, check + // for new compaction possibilities. We will still do this if they all errored out or + // the cfs is no longer valid, which is not helpful but not a problem. + submitBackground(cfs); } - finally + else { compactingCF.remove(cfs); } - if (ranCompaction) // only submit background if we actually ran a compaction - otherwise we end up in an infinite loop submitting noop background tasks - submitBackground(cfs); } - boolean maybeRunUpgradeTask(CompactionStrategyManager strategy) + CompletableFuture maybeRunUpgradeTask() { logger.debug("Checking for upgrade tasks {}.{}", cfs.keyspace.getName(), cfs.getTableName()); - try + if (currentlyBackgroundUpgrading.incrementAndGet() <= DatabaseDescriptor.maxConcurrentAutoUpgradeTasks()) { - if (currentlyBackgroundUpgrading.incrementAndGet() <= DatabaseDescriptor.maxConcurrentAutoUpgradeTasks()) + AbstractCompactionTask upgradeTask = findUpgradeSSTableTask(); + if (upgradeTask != null) { - AbstractCompactionTask upgradeTask = strategy.findUpgradeSSTableTask(); - if (upgradeTask != null) - { - upgradeTask.execute(active); - return true; - } + return CompletableFuture.runAsync(() -> upgradeTask.execute(active), executor) + .handle((ignored1, ignored2) -> { + currentlyBackgroundUpgrading.decrementAndGet(); + return null; + }); + } + else + { + logger.trace("No tasks available"); + currentlyBackgroundUpgrading.decrementAndGet(); } } - finally + else { currentlyBackgroundUpgrading.decrementAndGet(); } - logger.trace("No tasks available"); - return false; + return null; + } + + /** + * finds the oldest (by modification date) non-latest-version sstable on disk and creates an upgrade task for it + * @return + */ + @VisibleForTesting + @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute + public AbstractCompactionTask findUpgradeSSTableTask() + { + if (cfs.isAutoCompactionDisabled() || !DatabaseDescriptor.automaticSSTableUpgrade()) + return null; + + Set compacting = cfs.getTracker().getCompacting(); + List potentialUpgrade = cfs.getLiveSSTables() + .stream() + .filter(s -> !compacting.contains(s) && !s.descriptor.version.isLatestVersion()) + .sorted((o1, o2) -> { + File f1 = new File(o1.descriptor.filenameFor(Component.DATA)); + File f2 = new File(o2.descriptor.filenameFor(Component.DATA)); + return Longs.compare(f1.lastModified(), f2.lastModified()); + }).collect(Collectors.toList()); + for (SSTableReader sstable : potentialUpgrade) + { + LifecycleTransaction txn = cfs.getTracker().tryModify(sstable, OperationType.UPGRADE_SSTABLES); + if (txn != null) + { + logger.debug("Running automatic sstable upgrade for {}", sstable); + return cfs.getCompactionStrategy().createCompactionTask(txn, Integer.MIN_VALUE, Long.MAX_VALUE); + } + } + return null; } } @@ -503,7 +616,7 @@ public Iterable filterSSTables(LifecycleTransaction transaction) @Override public void execute(LifecycleTransaction txn) { - AbstractCompactionTask task = cfs.getCompactionStrategyManager().getCompactionTask(txn, NO_GC, Long.MAX_VALUE); + AbstractCompactionTask task = cfs.getCompactionStrategy().createCompactionTask(txn, NO_GC, Long.MAX_VALUE); task.setUserDefined(true); task.setCompactionType(OperationType.UPGRADE_SSTABLES); task.execute(active); @@ -583,7 +696,7 @@ public AllSSTableOpStatus performGarbageCollection(final ColumnFamilyStore cfSto public Iterable filterSSTables(LifecycleTransaction transaction) { Iterable originals = transaction.originals(); - if (cfStore.getCompactionStrategyManager().onlyPurgeRepairedTombstones()) + if (cfStore.onlyPurgeRepairedTombstones()) originals = Iterables.filter(originals, SSTableReader::isRepaired); List sortedSSTables = Lists.newArrayList(originals); Collections.sort(sortedSSTables, SSTableReader.maxTimestampAscending); @@ -591,25 +704,13 @@ public Iterable filterSSTables(LifecycleTransaction transaction) } @Override - public void execute(LifecycleTransaction txn) throws IOException + public void execute(LifecycleTransaction txn) { logger.debug("Garbage collecting {}", txn.originals()); - CompactionTask task = new CompactionTask(cfStore, txn, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds())) - { - @Override - protected CompactionController getCompactionController(Set toCompact) - { - return new CompactionController(cfStore, toCompact, gcBefore, null, tombstoneOption); - } - - @Override - protected int getLevel() - { - return txn.onlyOne().getSSTableLevel(); - } - }; - task.setUserDefined(true); - task.setCompactionType(OperationType.GARBAGE_COLLECT); + AbstractCompactionTask task = CompactionTask.forGarbageCollection(cfStore, + txn, + getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds()), + tombstoneOption); task.execute(active); } }, jobs, OperationType.GARBAGE_COLLECT); @@ -658,7 +759,7 @@ public Iterable filterSSTables(LifecycleTransaction transaction) public Map> groupByDiskIndex(Set needsRelocation) { - return needsRelocation.stream().collect(Collectors.groupingBy((s) -> diskBoundaries.getDiskIndex(s))); + return needsRelocation.stream().collect(Collectors.groupingBy((s) -> diskBoundaries.getDiskIndexFromKey(s))); } private boolean inCorrectLocation(SSTableReader sstable) @@ -675,7 +776,7 @@ private boolean inCorrectLocation(SSTableReader sstable) public void execute(LifecycleTransaction txn) { logger.debug("Relocating {}", txn.originals()); - AbstractCompactionTask task = cfs.getCompactionStrategyManager().getCompactionTask(txn, NO_GC, Long.MAX_VALUE); + AbstractCompactionTask task = cfs.getCompactionStrategy().createCompactionTask(txn, NO_GC, Long.MAX_VALUE); task.setUserDefined(true); task.setCompactionType(OperationType.RELOCATE); task.execute(active); @@ -740,7 +841,7 @@ private static void mutateFullyContainedSSTables(ColumnFamilyStore cfs, Set fullyContainedSSTables = findSSTablesToAnticompact(sstableIterator, normalizedRanges, sessionID); cfs.metric.bytesMutatedAnticompaction.inc(SSTableReader.getTotalBytes(fullyContainedSSTables)); - cfs.getCompactionStrategyManager().mutateRepaired(fullyContainedSSTables, UNREPAIRED_SSTABLE, sessionID, isTransient); + cfs.mutateRepaired(fullyContainedSSTables, UNREPAIRED_SSTABLE, sessionID, isTransient); // since we're just re-writing the sstable metdata for the fully contained sstables, we don't want // them obsoleted when the anti-compaction is complete. So they're removed from the transaction here txn.cancel(fullyContainedSSTables); @@ -852,13 +953,19 @@ public void performMaximal(final ColumnFamilyStore cfStore, boolean splitOutput) FBUtilities.waitOnFutures(submitMaximal(cfStore, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds()), splitOutput)); } - @SuppressWarnings("resource") // the tasks are executed in parallel on the executor, making sure that they get closed public List> submitMaximal(final ColumnFamilyStore cfStore, final int gcBefore, boolean splitOutput) + { + return submitMaximal(cfStore, gcBefore, splitOutput, active); + } + + @VisibleForTesting + @SuppressWarnings("resource") // the tasks are executed in parallel on the executor, making sure that they get closed + public List> submitMaximal(final ColumnFamilyStore cfStore, final int gcBefore, boolean splitOutput, TableOperationObserver obs) { // here we compute the task off the compaction executor, so having that present doesn't // confuse runWithCompactionsDisabled -- i.e., we don't want to deadlock ourselves, waiting // for ourselves to finish/acknowledge cancellation before continuing. - CompactionTasks tasks = cfStore.getCompactionStrategyManager().getMaximalTasks(gcBefore, splitOutput); + CompactionTasks tasks = cfStore.getCompactionStrategy().getMaximalTasks(gcBefore, splitOutput); if (tasks.isEmpty()) return Collections.emptyList(); @@ -875,7 +982,7 @@ public List> submitMaximal(final ColumnFamilyStore cfStore, final int { protected void runMayThrow() { - task.execute(active); + task.execute(obs); } }; @@ -898,7 +1005,7 @@ public void forceCompactionForTokenRange(ColumnFamilyStore cfStore, Collection refs = Refs.ref(Collections.singleton(sstable)); - CompactionIterator ci = new CompactionIterator(OperationType.CLEANUP, Collections.singletonList(scanner), controller, nowInSec, UUIDGen.getTimeUUID(), active)) + CompactionIterator ci = new CompactionIterator(OperationType.CLEANUP, Collections.singletonList(scanner), controller, nowInSec, UUIDGen.getTimeUUID())) { StatsMetadata metadata = sstable.getSSTableMetadata(); writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, metadata.repairedAt, metadata.pendingRepair, metadata.isTransient, sstable, txn)); @@ -1269,9 +1360,8 @@ private void doCleanupOne(final ColumnFamilyStore cfs, long bytesScanned = scanner.getBytesScanned(); - compactionRateLimiterAcquire(limiter, bytesScanned, lastBytesScanned, compressionRatio); - - lastBytesScanned = bytesScanned; + if (compactionRateLimiterAcquire(limiter, bytesScanned, lastBytesScanned, compressionRatio)) + lastBytesScanned = bytesScanned; } } @@ -1296,9 +1386,21 @@ private void doCleanupOne(final ColumnFamilyStore cfs, } - static void compactionRateLimiterAcquire(RateLimiter limiter, long bytesScanned, long lastBytesScanned, double compressionRatio) + static boolean compactionRateLimiterAcquire(RateLimiter limiter, long bytesScanned, long lastBytesScanned, double compressionRatio) { + if (DatabaseDescriptor.getCompactionThroughputMbPerSec() == 0) + return false; + long lengthRead = (long) ((bytesScanned - lastBytesScanned) * compressionRatio) + 1; + // Acquire at 128k granularity. At worst we'll exceed the limit a bit, but acquire is quite expensive. + if (lengthRead < ACQUIRE_GRANULARITY) + return false; + + return actuallyAcquire(limiter, lengthRead); + } + + private static boolean actuallyAcquire(RateLimiter limiter, long lengthRead) + { while (lengthRead >= Integer.MAX_VALUE) { limiter.acquire(Integer.MAX_VALUE); @@ -1308,6 +1410,7 @@ static void compactionRateLimiterAcquire(RateLimiter limiter, long bytesScanned, { limiter.acquire((int) lengthRead); } + return true; } private static abstract class CleanupStrategy @@ -1429,7 +1532,7 @@ public static SSTableWriter createWriter(ColumnFamilyStore cfs, isTransient, sstable.getSSTableLevel(), sstable.header, - cfs.indexManager.listIndexes(), + cfs.indexManager.listIndexGroups(), txn); } @@ -1466,7 +1569,7 @@ public static SSTableWriter createWriterForAntiCompaction(ColumnFamilyStore cfs, cfs.metadata, new MetadataCollector(sstables, cfs.metadata().comparator, minLevel), SerializationHeader.make(cfs.metadata(), sstables), - cfs.indexManager.listIndexes(), + cfs.indexManager.listIndexGroups(), txn); } @@ -1499,7 +1602,7 @@ private void doAntiCompaction(ColumnFamilyStore cfs, // make use of any actual repairedAt value and splitting up sstables just for that is not worth it at this point. Set unrepairedSSTables = sstables.stream().filter((s) -> !s.isRepaired()).collect(Collectors.toSet()); cfs.metric.bytesAnticompacted.inc(SSTableReader.getTotalBytes(unrepairedSSTables)); - Collection> groupedSSTables = cfs.getCompactionStrategyManager().groupSSTablesForAntiCompaction(unrepairedSSTables); + Collection> groupedSSTables = cfs.getCompactionStrategy().groupSSTablesForAntiCompaction(unrepairedSSTables); // iterate over sstables to check if the full / transient / unrepaired ranges intersect them. int antiCompactedSSTableCount = 0; @@ -1579,52 +1682,55 @@ public void obsoleteOriginals() {} public void close() {} } - CompactionStrategyManager strategy = cfs.getCompactionStrategyManager(); + CompactionStrategy strategy = cfs.getCompactionStrategy(); try (SharedTxn sharedTxn = new SharedTxn(txn); SSTableRewriter fullWriter = SSTableRewriter.constructWithoutEarlyOpening(sharedTxn, false, groupMaxDataAge); SSTableRewriter transWriter = SSTableRewriter.constructWithoutEarlyOpening(sharedTxn, false, groupMaxDataAge); SSTableRewriter unrepairedWriter = SSTableRewriter.constructWithoutEarlyOpening(sharedTxn, false, groupMaxDataAge); - AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(txn.originals()); + ScannerList scanners = strategy.getScanners(txn.originals()); CompactionController controller = new CompactionController(cfs, sstableAsSet, getDefaultGcBefore(cfs, nowInSec)); - CompactionIterator ci = getAntiCompactionIterator(scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID(), active, isCancelled)) + CompactionIterator ci = getAntiCompactionIterator(scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID(), isCancelled)) { - int expectedBloomFilterSize = Math.max(cfs.metadata().params.minIndexInterval, (int)(SSTableReader.getApproximateKeyCount(sstableAsSet))); - - fullWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, pendingRepair, false, sstableAsSet, txn)); - transWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, pendingRepair, true, sstableAsSet, txn)); - unrepairedWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, NO_PENDING_REPAIR, false, sstableAsSet, txn)); + TableOperation op = ci.getOperation(); + try (NonThrowingCloseable cls = active.onOperationStart(op)) + { + int expectedBloomFilterSize = Math.max(cfs.metadata().params.minIndexInterval, (int)(SSTableReader.getApproximateKeyCount(sstableAsSet))); - Predicate fullChecker = !ranges.onlyFull().isEmpty() ? new Range.OrderedRangeContainmentChecker(ranges.onlyFull().ranges()) : t -> false; - Predicate transChecker = !ranges.onlyTransient().isEmpty() ? new Range.OrderedRangeContainmentChecker(ranges.onlyTransient().ranges()) : t -> false; - double compressionRatio = scanners.getCompressionRatio(); - if (compressionRatio == MetadataCollector.NO_COMPRESSION_RATIO) - compressionRatio = 1.0; + fullWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, pendingRepair, false, sstableAsSet, txn)); + transWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, pendingRepair, true, sstableAsSet, txn)); + unrepairedWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, UNREPAIRED_SSTABLE, NO_PENDING_REPAIR, false, sstableAsSet, txn)); - long lastBytesScanned = 0; + Predicate fullChecker = !ranges.onlyFull().isEmpty() ? new Range.OrderedRangeContainmentChecker(ranges.onlyFull().ranges()) : t -> false; + Predicate transChecker = !ranges.onlyTransient().isEmpty() ? new Range.OrderedRangeContainmentChecker(ranges.onlyTransient().ranges()) : t -> false; + double compressionRatio = scanners.getCompressionRatio(); + if (compressionRatio == MetadataCollector.NO_COMPRESSION_RATIO) + compressionRatio = 1.0; - while (ci.hasNext()) - { - try (UnfilteredRowIterator partition = ci.next()) + long lastBytesScanned = 0; + while (ci.hasNext()) { - Token token = partition.partitionKey().getToken(); - // if this row is contained in the full or transient ranges, append it to the appropriate sstable - if (fullChecker.test(token)) - { - fullWriter.append(partition); - } - else if (transChecker.test(token)) - { - transWriter.append(partition); - } - else + try (UnfilteredRowIterator partition = ci.next()) { - // otherwise, append it to the unrepaired sstable - unrepairedWriter.append(partition); + Token token = partition.partitionKey().getToken(); + // if this row is contained in the full or transient ranges, append it to the appropriate sstable + if (fullChecker.test(token)) + { + fullWriter.append(partition); + } + else if (transChecker.test(token)) + { + transWriter.append(partition); + } + else + { + // otherwise, append it to the unrepaired sstable + unrepairedWriter.append(partition); + } + long bytesScanned = scanners.getTotalBytesScanned(); + if (compactionRateLimiterAcquire(limiter, bytesScanned, lastBytesScanned, compressionRatio)) + lastBytesScanned = bytesScanned; } - long bytesScanned = scanners.getTotalBytesScanned(); - compactionRateLimiterAcquire(limiter, bytesScanned, lastBytesScanned, compressionRatio); - lastBytesScanned = bytesScanned; } } @@ -1670,33 +1776,53 @@ else if (transChecker.test(token)) } @VisibleForTesting - public static CompactionIterator getAntiCompactionIterator(List scanners, CompactionController controller, int nowInSec, UUID timeUUID, ActiveCompactionsTracker activeCompactions, BooleanSupplier isCancelled) + public static CompactionIterator getAntiCompactionIterator(List scanners, CompactionController controller, int nowInSec, UUID timeUUID, BooleanSupplier isCancelled) { - return new CompactionIterator(OperationType.ANTICOMPACTION, scanners, controller, nowInSec, timeUUID, activeCompactions) { + return new CompactionIterator(OperationType.ANTICOMPACTION, scanners, controller, nowInSec, timeUUID) { + @Override + public TableOperation createOperation() + { + return getAntiCompactionOperation(super.createOperation(), isCancelled); + } + }; + } + + @VisibleForTesting + public static TableOperation getAntiCompactionOperation(TableOperation compaction, BooleanSupplier isCancelled) + { + return new AbstractTableOperation() + { + @Override + public boolean isGlobal() + { + return false; + } + + @Override + public OperationProgress getProgress() + { + return compaction.getProgress(); + } + @Override public boolean isStopRequested() { - return super.isStopRequested() || isCancelled.getAsBoolean(); + return compaction.isStopRequested() || isCancelled.getAsBoolean(); } }; } @VisibleForTesting - ListenableFuture submitIndexBuild(final SecondaryIndexBuilder builder, ActiveCompactionsTracker activeCompactions) + ListenableFuture submitIndexBuild(final SecondaryIndexBuilder builder, TableOperationObserver activeCompactions) { Runnable runnable = new Runnable() { public void run() { - activeCompactions.beginCompaction(builder); - try + try (NonThrowingCloseable c = activeCompactions.onOperationStart(builder)) { builder.build(); } - finally - { - activeCompactions.finishCompaction(builder); - } } }; @@ -1716,7 +1842,7 @@ public Future submitCacheWrite(final AutoSavingCache.Writer writer) return submitCacheWrite(writer, active); } - Future submitCacheWrite(final AutoSavingCache.Writer writer, ActiveCompactionsTracker activeCompactions) + Future submitCacheWrite(final AutoSavingCache.Writer writer, TableOperationObserver activeCompactions) { Runnable runnable = new Runnable() { @@ -1724,20 +1850,15 @@ public void run() { if (!AutoSavingCache.flushInProgress.add(writer.cacheType())) { - logger.trace("Cache flushing was already in progress: skipping {}", writer.getCompactionInfo()); + logger.trace("Cache flushing was already in progress: skipping {}", writer.getProgress()); return; } try { - activeCompactions.beginCompaction(writer); - try + try (NonThrowingCloseable c = activeCompactions.onOperationStart(writer)) { writer.saveCache(); } - finally - { - activeCompactions.finishCompaction(writer); - } } finally { @@ -1755,17 +1876,12 @@ public List runIndexSummaryRedistribution(IndexSummaryRedistribut } @VisibleForTesting - List runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution, ActiveCompactionsTracker activeCompactions) throws IOException + List runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution, TableOperationObserver activeCompactions) throws IOException { - activeCompactions.beginCompaction(redistribution); - try + try(Closeable c = activeCompactions.onOperationStart(redistribution)) { return redistribution.redistributeSummaries(); } - finally - { - activeCompactions.finishCompaction(redistribution); - } } public static int getDefaultGcBefore(ColumnFamilyStore cfs, int nowInSec) @@ -1781,24 +1897,19 @@ public ListenableFuture submitViewBuilder(final ViewBuilderTask task) } @VisibleForTesting - ListenableFuture submitViewBuilder(final ViewBuilderTask task, ActiveCompactionsTracker activeCompactions) + ListenableFuture submitViewBuilder(final ViewBuilderTask task, TableOperationObserver activeCompactions) { return viewBuildExecutor.submitIfRunning(() -> { - activeCompactions.beginCompaction(task); - try + try(Closeable c = activeCompactions.onOperationStart(task)) { return task.call(); } - finally - { - activeCompactions.finishCompaction(task); - } }, "view build"); } public int getActiveCompactions() { - return active.getCompactions().size(); + return active.getTableOperations().size(); } static class CompactionExecutor extends JMXEnabledThreadPoolExecutor @@ -1959,22 +2070,32 @@ public void incrementSstablesDropppedFromCompactions(long num) metrics.sstablesDropppedFromCompactions.inc(num); } + public void incrementRemovedExpiredSSTables(long num) + { + metrics.removedExpiredSSTables.mark(num); + } + + public void incrementDeleteOnlyCompactions() + { + metrics.deleteOnlyCompactions.mark(); + } + public List> getCompactions() { - List compactionHolders = active.getCompactions(); - List> out = new ArrayList>(compactionHolders.size()); - for (CompactionInfo.Holder ci : compactionHolders) - out.add(ci.getCompactionInfo().asMap()); + List operationSources = active.getTableOperations(); + List> out = new ArrayList>(operationSources.size()); + for (TableOperation op : operationSources) + out.add(op.getProgress().asMap()); return out; } public List getCompactionSummary() { - List compactionHolders = active.getCompactions(); - List out = new ArrayList(compactionHolders.size()); - for (CompactionInfo.Holder ci : compactionHolders) - out.add(ci.getCompactionInfo().toString()); + List operationSources = active.getTableOperations(); + List out = new ArrayList(operationSources.size()); + for (TableOperation ci : operationSources) + out.add(ci.getProgress().toString()); return out; } @@ -2013,20 +2134,20 @@ public long getCompletedTasks() public void stopCompaction(String type) { OperationType operation = OperationType.valueOf(type); - for (Holder holder : active.getCompactions()) + for (TableOperation operationSource : active.getTableOperations()) { - if (holder.getCompactionInfo().getTaskType() == operation) - holder.stop(); + if (operationSource.getProgress().operationType() == operation) + operationSource.stop(); } } public void stopCompactionById(String compactionId) { - for (Holder holder : active.getCompactions()) + for (TableOperation operationSource : active.getTableOperations()) { - UUID holderId = holder.getCompactionInfo().getTaskId(); + UUID holderId = operationSource.getProgress().operationId(); if (holderId != null && holderId.equals(UUID.fromString(compactionId))) - holder.stop(); + operationSource.stop(); } } @@ -2166,6 +2287,60 @@ public void setMaxConcurrentAutoUpgradeTasks(int value) } } + /** + * Try to stop all of the compactions for given tables. + * + * Note that this method does not wait for all compactions to finish; you'll need to loop against + * isCompacting if you want that behavior. + * + * @param tables The tables to try to stop compaction upon. + * @param opPredicate Predicate to define which compaction operation to stop, based on its type. + * @param readerPredicate Predicate to define which compaction to stop based on candidate sstables. + * @param waitForInterruption whether to wait until interrupted compaction has fully stopped + * + * @return True if any compaction has been interrupted false otherwise. + */ + public boolean interruptCompactionFor(Iterable tables, Predicate opPredicate, Predicate readerPredicate, + boolean waitForInterruption) + { + assert tables != null; + + // interrupt in-progress compactions + Set interrupted = new HashSet<>(); + for (TableOperation operationSource : active.getTableOperations()) + { + AbstractTableOperation.OperationProgress info = operationSource.getProgress(); + + if (Iterables.contains(tables, info.metadata()) && opPredicate.test(info.operationType())) + { + operationSource.stop(); + interrupted.add(operationSource); + } + } + + if (waitForInterruption) + { + // wait at most 2 minutes + long start = System.nanoTime(); + long wait = TimeUnit.MINUTES.toNanos(2); + + for (TableOperation operation : interrupted) + { + while (active.isActive(operation) && System.nanoTime() - start < wait) + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); + + if (active.isActive(operation)) + throw new RuntimeException(String.format("Compaction task (%s) didn't finish within 2 minutes", operation.getProgress())); + } + } + + return !interrupted.isEmpty(); + } + + public boolean interruptCompactionFor(Iterable columnFamilies, Predicate sstablePredicate, boolean interruptValidation) + { + return interruptCompactionFor(columnFamilies, sstablePredicate, interruptValidation, AbstractTableOperation.StopTrigger.NONE); + } /** * Try to stop all of the compactions for given ColumnFamilies. * @@ -2175,33 +2350,49 @@ public void setMaxConcurrentAutoUpgradeTasks(int value) * @param columnFamilies The ColumnFamilies to try to stop compaction upon. * @param sstablePredicate the sstable predicate to match on * @param interruptValidation true if validation operations for repair should also be interrupted + * @return True if any compaction has been interrupted false otherwise. */ - public void interruptCompactionFor(Iterable columnFamilies, Predicate sstablePredicate, boolean interruptValidation) + public boolean interruptCompactionFor(Iterable columnFamilies, Predicate sstablePredicate, boolean interruptValidation, AbstractTableOperation.StopTrigger trigger) { assert columnFamilies != null; // interrupt in-progress compactions - for (Holder compactionHolder : active.getCompactions()) + boolean interrupted = false; + for (TableOperation operationSource : active.getTableOperations()) { - CompactionInfo info = compactionHolder.getCompactionInfo(); - if ((info.getTaskType() == OperationType.VALIDATION) && !interruptValidation) + AbstractTableOperation.OperationProgress info = operationSource.getProgress(); + if ((info.operationType() == OperationType.VALIDATION) && !interruptValidation) continue; - if (info.getTableMetadata() == null || Iterables.contains(columnFamilies, info.getTableMetadata())) + if (info.metadata() == null || Iterables.contains(columnFamilies, info.metadata())) { - if (info.shouldStop(sstablePredicate)) - compactionHolder.stop(); + if (operationSource.shouldStop(sstablePredicate)) + { + operationSource.stop(trigger); + interrupted = true; + } } } + return interrupted; + } + + public boolean interruptCompactionFor(Iterable tables) + { + return interruptCompactionFor(tables, Predicates.alwaysTrue(), true); } public void interruptCompactionForCFs(Iterable cfss, Predicate sstablePredicate, boolean interruptValidation) + { + interruptCompactionForCFs(cfss, sstablePredicate, interruptValidation, AbstractTableOperation.StopTrigger.NONE); + } + + public void interruptCompactionForCFs(Iterable cfss, Predicate sstablePredicate, boolean interruptValidation, AbstractTableOperation.StopTrigger trigger) { List metadata = new ArrayList<>(); for (ColumnFamilyStore cfs : cfss) metadata.add(cfs.metadata()); - interruptCompactionFor(metadata, sstablePredicate, interruptValidation); + interruptCompactionFor(metadata, sstablePredicate, interruptValidation, trigger); } public void waitForCessation(Iterable cfss, Predicate sstablePredicate) @@ -2219,14 +2410,14 @@ public void waitForCessation(Iterable cfss, Predicate getSSTableTasks() + public List getSSTableTasks() { - return active.getCompactions() + return active.getTableOperations() .stream() - .map(CompactionInfo.Holder::getCompactionInfo) - .filter(task -> task.getTaskType() != OperationType.COUNTER_CACHE_SAVE - && task.getTaskType() != OperationType.KEY_CACHE_SAVE - && task.getTaskType() != OperationType.ROW_CACHE_SAVE) + .map(TableOperation::getProgress) + .filter(progress -> progress.operationType() != OperationType.COUNTER_CACHE_SAVE + && progress.operationType() != OperationType.KEY_CACHE_SAVE + && progress.operationType() != OperationType.ROW_CACHE_SAVE) .collect(Collectors.toList()); } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionObserver.java b/src/java/org/apache/cassandra/db/compaction/CompactionObserver.java new file mode 100644 index 000000000000..286e3fb9e3a0 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionObserver.java @@ -0,0 +1,45 @@ +/* + * 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.compaction; + +import java.util.UUID; + +/** + * An observer of a compaction operation. It is notified when a compaction operation is started. + *

    + * It returns a closeable that is invoked when the compaction is finished. + *

    + * The progress can be queried at any time to obtain real-time updates of the compaction operation. + */ +public interface CompactionObserver +{ + /** + * Indicates that a compaction has started. + *

    + * @param progress the compaction progress, it contains the unique id and real-time progress information + */ + void onInProgress(CompactionProgress progress); + + /** + * Indicates that a compaction with the given id has completed. + *

    + * @param id the id of the compaction + */ + void onCompleted(UUID id); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionPick.java b/src/java/org/apache/cassandra/db/compaction/CompactionPick.java new file mode 100644 index 000000000000..4b8881167747 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionPick.java @@ -0,0 +1,244 @@ +/* + * 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.compaction; + +import java.util.Collection; +import java.util.Collections; +import java.util.Objects; +import java.util.UUID; +import java.util.stream.Collectors; + +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableSet; +import org.apache.cassandra.io.sstable.format.SSTableReader; + +/** + * A set of sstables that were picked for compaction along with some other relevant properties. + *

    + * This is a list of sstables that should be compacted together after having been picked by a compaction strategy, + * for example from a bucket in {@link SizeTieredCompactionStrategy} or from a level in {@link LeveledCompactionStrategy}. + * Also, it contains other useful parameters such as a score that was assigned to this candidate (the read hotness or level + * score depending on the strategy) and the level, if applicable. + **/ +class CompactionPick +{ + final static CompactionPick EMPTY = create(-1, Collections.emptyList(), 0); + + /** The key to the parent compaction aggregate, e.g. a level number or tier avg size, -1 if no parent */ + final long parent; + + /** The sstables to be compacted */ + final ImmutableSet sstables; + + /** Only expired sstables */ + final ImmutableSet expired; + + /** The sum of all the sstable hotness scores */ + final double hotness; + + /** The average size in bytes for the sstables in this compaction */ + final long avgSizeInBytes; + + /** The total size on disk for the sstables in this compaction */ + final long totSizeInBytes; + + /** The unique compaction id, this is only available when a compaction is submitted */ + @Nullable + volatile UUID id; + + /** The compaction progress, this is only available when compaction actually starts and will be null as long as + * the candidate is still pending execution, also some tasks cannot report a progress at all, e.g. {@link SingleSSTableLCSTask}. + * */ + @Nullable volatile CompactionProgress progress; + + /** Set to true when the compaction has completed */ + volatile boolean completed; + + private CompactionPick(long parent, + Collection compacting, + Collection expired, + double hotness, + long avgSizeInBytes, + long totSizeInBytes) + { + this.parent = parent; + this.sstables = ImmutableSet.copyOf(compacting); + this.expired = ImmutableSet.copyOf(expired); + this.hotness = hotness; + this.avgSizeInBytes = avgSizeInBytes; + this.totSizeInBytes = totSizeInBytes; + } + + /** + * Create a pending compaction candidate calculating hotness and avg size. + */ + static CompactionPick create(long parent, Collection sstables, Collection expired) + { + Collection nonExpiring = sstables.stream().filter(sstable -> !expired.contains(sstable)).collect(Collectors.toList()); + return create(parent, + sstables, + expired, + CompactionAggregate.getTotHotness(nonExpiring), + CompactionAggregate.getAvgSizeBytes(nonExpiring), + CompactionAggregate.getTotSizeBytes(nonExpiring)); + } + + static CompactionPick create(long parent, Collection sstables) + { + return create(parent, sstables, Collections.emptyList()); + } + + /** + * Create a pending compaction candidate calculating avg size. + */ + static CompactionPick create(long parent, Collection sstables, double hotness) + { + return create(parent, sstables, Collections.emptyList(), hotness, CompactionAggregate.getAvgSizeBytes(sstables), CompactionAggregate.getTotSizeBytes(sstables)); + } + + /** + * Create a pending compaction candidate with the given parameters. + */ + static CompactionPick create(long parent, Collection sstables, Collection expired, double hotness, long avgSizeInBytes, long totSizeInBytes) + { + return new CompactionPick(parent, sstables, expired, hotness, avgSizeInBytes, totSizeInBytes); + } + + /** + * Create new compaction pick similar to the one provided but with a new parent. + */ + static CompactionPick create(long parent, CompactionPick pick) + { + return new CompactionPick(parent, pick.sstables, pick.expired, pick.hotness, pick.avgSizeInBytes, pick.totSizeInBytes); + } + + public double hotness() + { + return hotness; + } + + public long avgSizeInBytes() + { + return avgSizeInBytes; + } + + void setSubmitted(UUID id) + { + if (id == null) + throw new IllegalArgumentException("Id cannot be null"); + + if (this.id != null) + throw new IllegalStateException("Already submitted"); + + this.id = id; + } + /** + * Set the compaction progress, this means the compaction pick has started executing. + */ + void setProgress(CompactionProgress progress) + { + if (progress == null) + throw new IllegalArgumentException("Progress cannot be null"); + + if (this.progress != null) + throw new IllegalStateException("Already compacting"); + + if (this.id == null) + setSubmitted(progress.operationId()); + else if (this.id != progress.operationId()) + throw new IllegalStateException("Submitted with a different id"); + + this.progress = progress; + } + + void setCompleted() + { + if (this.completed) + throw new IllegalStateException("Already completed"); + + this.completed = true; + } + + /** + * Add more sstables to the collection of sstables initially picked. + *

    + * This is currently used by {@link TimeWindowCompactionStrategy} to add expired sstables. + * + * @param expired the sstables to add + */ + CompactionPick withExpiredSSTables(Collection expired) + { + ImmutableSet newSSTables = ImmutableSet.builder() + .addAll(this.sstables) + .addAll(expired) + .build(); + ImmutableSet newExpired = ImmutableSet.builder() + .addAll(this.expired) + .addAll(expired) + .build(); + return new CompactionPick(parent, + newSSTables, + newExpired, + hotness, + avgSizeInBytes, + totSizeInBytes); + } + + /** + * @return true if this compaction candidate is empty, that is it has no sstables to compact. + */ + boolean isEmpty() + { + return sstables.isEmpty(); + } + + boolean hasExpiredOnly() + { + return sstables.size() == expired.size(); + } + + @Override + public int hashCode() + { + return Objects.hash(parent, sstables, expired); + } + + @Override + public boolean equals(Object obj) + { + if (obj == this) + return true; + + if (!(obj instanceof CompactionPick)) + return false; + + CompactionPick that = (CompactionPick) obj; + + // a pick is the same if the sstables are the same given that the other properties are derived from sstables and two + // picks are the same whether compaction has started or not so the progress and completed properties should not determine equality + return parent == that.parent && sstables.equals(that.sstables) && expired.equals(that.expired); + } + + @Override + public String toString() + { + return String.format("Parent: %d, Hotness: %f, Avg size in bytes: %d, id: %s, sstables: %s, expired: %s", parent, hotness, avgSizeInBytes, id, sstables, expired); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionProgress.java b/src/java/org/apache/cassandra/db/compaction/CompactionProgress.java new file mode 100644 index 000000000000..08099ef785aa --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionProgress.java @@ -0,0 +1,131 @@ +/* + * 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.compaction; + +import java.util.Collection; + +import javax.annotation.Nullable; + +import org.apache.cassandra.io.sstable.format.SSTableReader; + +/** + * The progress information for a compaction operation. This adds compaction + * specific information to {@link TableOperation.Progress}. + */ +public interface CompactionProgress extends TableOperation.Progress +{ + /** + * The compaction strategy if available, otherwise null. + *

    + * The compaction strategy may not be available for some operations that use compaction task such + * as GC or sstable splitting. + * + * @return the compaction strategy when available or null. + */ + @Nullable + CompactionStrategy strategy(); + + /** + * @return true if the compaction was requested to interrupt + */ + boolean isStopRequested(); + + /** + * @return input sstables + */ + Collection inSSTables(); + + /** + * @return output sstables + */ + Collection outSSTables(); + + /** + * @return Size on disk (compressed) of the input sstables. + */ + long inputDiskSize(); + + /** + * @return The uncompressed size of the input sstables. + */ + long inputUncompressedSize(); + + /** Same as {@link this#inputDiskSize()} except for LCS where it estimates + * the compressed size for number of keys that will be read from the input sstables, + * see {@link org.apache.cassandra.db.compaction.LeveledCompactionStrategy}. */ + long adjustedInputDiskSize(); + + /** + * @return Size on disk (compressed) of the output sstables. + */ + long outputDiskSize(); + + /** + * @return the number of bytes processed by the compaction iterator. For compressed or encrypted sstables, + * this is the number of bytes processed by the iterator after decompression, so this is the current + * position in the uncompressed sstable files. + */ + long uncompressedBytesRead(); + + /** + * @return the number of bytes processed by the compaction iterator for sstables on the specified level. + * For compressed or encrypted sstables, this is the number of bytes processed by the iterator after decompression, + * so this is the current position in the uncompressed sstable files. + */ + long uncompressedBytesRead(int level); + + /** + * @return the number of bytes that were written before compression is applied (uncompressed size). + */ + long uncompressedBytesWritten(); + + /** + * @return the duration so far in nanoseconds. + */ + long durationInNanos(); + + /** + * @return total number of partitions read + */ + long partitionsRead(); + + /** + * @return otal number of rows read + */ + long rowsRead(); + + /** + * The partitions histogram maps the number of sstables to the number of partitions that were merged with that number of input sstables. + * + * @return the partitions histogram + */ + long[] partitionsHistogram(); + + /** + * The rows histogram maps the number of sstables to the number of rows that were merged with that number of input sstables. + * + * @return the rows histogram + */ + long[] rowsHistogram(); + + /** + * @return the ratio of bytes before and after compaction, using the adjusted input and output disk sizes (uncompressed values). + */ + double sizeRatio(); +} diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategy.java new file mode 100644 index 000000000000..d687b125ac6c --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategy.java @@ -0,0 +1,188 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.ScannerList; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; + +/** + * The common interface between legacy compaction strategies (those that extend {@link LegacyAbstractCompactionStrategy} + * and the new compaction strategy, {@link UnifiedCompactionStrategy}. + */ +public interface CompactionStrategy extends CompactionObserver +{ + /** + * @return the compaction logger optionally logs events in a csv file. + */ + CompactionLogger getCompactionLogger(); + + /** + * For internal, temporary suspension of background compactions so that we can do exceptional + * things like truncate or major compaction + */ + void pause(); + + /** + * For internal, temporary suspension of background compactions so that we can do exceptional + * things like truncate or major compaction + */ + void resume(); + + /** + * Performs any extra initialization required + */ + void startup(); + + /** + * Releases any resources if this strategy is shutdown (when the CFS is reloaded after a schema change). + */ + void shutdown(); + + /** + * @param gcBefore throw away tombstones older than this + * + * @return the next background/minor compaction tasks to run; empty if nothing to do. + * + * Is responsible for marking its sstables as compaction-pending. + */ + Collection getNextBackgroundTasks(int gcBefore); + + /** + * @param gcBefore throw away tombstones older than this + * + * @return a compaction task that should be run to compact this columnfamilystore + * as much as possible. Null if nothing to do. + * + * Is responsible for marking its sstables as compaction-pending. + */ + @SuppressWarnings("resource") + CompactionTasks getMaximalTasks(int gcBefore, boolean splitOutput); + + /** + * @param sstables SSTables to compact. Must be marked as compacting. + * @param gcBefore throw away tombstones older than this + * + * @return a compaction task corresponding to the requested sstables. + * Will not be null. (Will throw if user requests an invalid compaction.) + * + * Is responsible for marking its sstables as compaction-pending. + */ + @SuppressWarnings("resource") + CompactionTasks getUserDefinedTasks(Collection sstables, int gcBefore); + + /** + * Get the estimated remaining compactions. + * + * @return the number of background tasks estimated to still be needed for this strategy + */ + int getEstimatedRemainingTasks(); + + /** + * Create a compaction task for the sstables in the transaction. + * + * @return a valid compaction task that can be executed. + */ + AbstractCompactionTask createCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes); + + /** + * @return the total number of background compactions, pending or in progress + */ + int getTotalCompactions(); + + /** + * Return the statistics. Not all strategies will provide non-empty statistics, + * the legacy strategies that do not support aggregates will return empty statistics. + *

    + * @return statistics about this compaction picks. + */ + List getStatistics(); + + /** + * @return size in bytes of the largest sstables for this strategy + */ + long getMaxSSTableBytes(); + + /** + * @return the number of sstables for each level, if this strategy supports levels. Otherwise return an empty array. + */ + int[] getSSTableCountPerLevel(); + + /** + * @return the level fanout size if applicable to this strategy. Otherwise return the default LCS fanout size. + */ + int getLevelFanoutSize(); + + /** + * Returns a list of KeyScanners given sstables and a range on which to scan. + * The default implementation simply grab one SSTableScanner per-sstable, but overriding this method + * allow for a more memory efficient solution if we know the sstable don't overlap (see + * LeveledCompactionStrategy for instance). + */ + ScannerList getScanners(Collection sstables, Collection> ranges); + + default ScannerList getScanners(Collection toCompact) + { + return getScanners(toCompact, null); + } + + /** + * @return the name of the strategy + */ + String getName(); + + /** + * Returns the sstables managed by the strategy + */ + Set getSSTables(); + + /** + * Group sstables that can be anti-compacted togetehr. + */ + Collection> groupSSTablesForAntiCompaction(Collection sstablesToGroup); + + /** + * Create an sstable writer that is suitable for the strategy. + */ + SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, + long keyCount, + long repairedAt, + UUID pendingRepair, + boolean isTransient, + MetadataCollector collector, + SerializationHeader header, + Collection indexGroups, + LifecycleNewTracker lifecycleNewTracker); + + /** + * @return true if the strategy supports early open + */ + boolean supportsEarlyOpen(); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyContainer.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyContainer.java new file mode 100644 index 000000000000..1ce0c37cb080 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyContainer.java @@ -0,0 +1,191 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.util.List; +import java.util.UUID; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.notifications.INotificationConsumer; +import org.apache.cassandra.schema.CompactionParams; + +/** + * A strategy container manages compaction strategies for a {@link ColumnFamilyStore}. + * + * This class is responsible for: + * - providing a single interface for possibly multiple active strategy instances - e.g. due to having + * multiple arenas for repaired, unrepaired, pending, transient SSTables. + * - updating or recreating the strategies when configuration change - e.g. compaction parameters + * or disk boundaries + */ +public interface CompactionStrategyContainer extends CompactionStrategy, INotificationConsumer +{ + /** + * Enable compaction. + */ + void enable(); + + /** + * Disable compaction. + */ + void disable(); + + /** + * @return {@code true} if compaction is enabled and running; e.g. if autocompaction has been disabled via nodetool + * or JMX, this should return {@code false}, even if the underlying compaction strategy hasn't been paused. + */ + boolean isEnabled(); + + /** + * @return {@code true} if compaction is running, i.e. if the underlying compaction strategy is not currently + * paused or being shut down. + */ + boolean isActive(); + + /** + * The reason for reloading + */ + enum ReloadReason + { + /** A new strategy container has been created. */ + FULL, + + /** A new strategy container has been reloaded due to table metadata changes, e.g. a schema change. */ + METADATA_CHANGE, + + /** A request over JMX to update the compaction parameters only locally, without changing the schema permanently. */ + JMX_REQUEST, + + /** The disk boundaries were updated, in this case the strategies may need to be recreated even if the params haven't changed */ + DISK_BOUNDARIES_UPDATED + } + + /** + * Reload the strategy container taking into account the state of the previous strategy container instance + * ({@code this}, in case we're not reloading after switching between containers), the new compaction parameters, + * and the reason for reloading. + *

    + * Depending on the reason, different actions are taken, for example the schema parameters are not updated over + * JMX and the decision on whether to enable or disable compaction depends only on the parameters over JMX, but + * also on the previous JMX directive in case of a full reload. Also, the disk boundaries are not updated over JMX. + *

    + * See the implementations of this method for more details. + * + * @param previous the strategy container instance which state needs to be inherited/taken into account, in many + * cases the same as {@code this}, but never {@code null}. + * @param compactionParams the new compaction parameters + * @param reason the reason for reloading + * + * @return existing or new container with updated parameters + */ + CompactionStrategyContainer reload(@Nonnull CompactionStrategyContainer previous, + CompactionParams compactionParams, + ReloadReason reason); + + /** + * @param params new compaction parameters + * @param reason the reason for reloading + * @return {@code true} if the compaction parameters should be updated on reload + */ + default boolean shouldReload(CompactionParams params, ReloadReason reason) + { + return reason != CompactionStrategyContainer.ReloadReason.METADATA_CHANGE || !params.equals(getMetadataCompactionParams()); + } + + /** + * Creates new {@link CompactionStrategyContainer} and loads its parameters + * + * This method is used by {@link CompactionStrategyFactory} to create a + * {@link CompactionStrategyContainer}s via reflection. + * + * @param previous the strategy container instance which state needs to be inherited/taken into account + * or {@code null} if there was no container to inherit from. + * @param strategyFactory the factory instance responsible for creating the CSM + * @param compactionParams the new compaction parameters + * @param reason the reason for creating a new container + * + * @return a new {@link CompactionStrategyContainer} with newly loaded parameters + */ + static CompactionStrategyContainer create(@Nullable CompactionStrategyContainer previous, + CompactionStrategyFactory strategyFactory, + CompactionParams compactionParams, + CompactionStrategyContainer.ReloadReason reason) + { + throw new UnsupportedOperationException("Implementations of CompactionStrategyContainer must implement static create method"); + } + + /** + * Return the compaction parameters. These are not necessarily the same as the ones specified in the schema, they + * may have been overwritten over JMX. + * + * @return the compaction params currently active + */ + CompactionParams getCompactionParams(); + + /** + * Returns the compaction parameters set via metadata. + * + * This method is useful to decide if we should update the compaction strategy due to a + * metadata change such as a schema changed caused by an ALTER TABLE. + * + * If a user changes the local compaction strategy via JMX and then later ALTERs a compaction parameter, + * we will use the new compaction parameters but we will not override the JMX parameters if compaction + * was not changed by the ALTER. + * + * @return the compaction parameters set via metadata changes + */ + CompactionParams getMetadataCompactionParams(); + + /** + * This method is to keep compatibility with strategies baked by {@link CompactionStrategyManager} where + * there are multiple inner strategies handling sstables by repair status. + * + * @return all inner compaction strategies + */ + List getStrategies(); + + /** + * This method is to keep compatibility with strategies baked by {@link CompactionStrategyManager} where + * there are multiple inner strategies handling sstables by repair status. + * + * Note that if {@code isRepaired} is true, {@code pendingRepair} must be null. + * + * @param isRepaired will return strategies for repaired SSTables; must be {@code false} if + * {@code pendingRepair} is specified + * @param pendingRepair will return strategies for the given pending repair; must be {@code null} + * if {@code isRepaired} is true + * + * @return a list of inner strategies that match given parameters + */ + List getStrategies(boolean isRepaired, @Nullable UUID pendingRepair); + + /** + * Called to clean up state when a repair session completes. + * + * @param sessionID repair session id. + */ + void repairSessionCompleted(UUID sessionID); + + /** + * The method is for CompactionStrategyManager to use with {@link org.apache.cassandra.db.ColumnFamilyStore#mutateRepaired}. + * UnifiedCompactionContainer does not need it. + */ + ReentrantReadWriteLock.WriteLock getWriteLock(); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyFactory.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyFactory.java new file mode 100644 index 000000000000..847e70c0d066 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyFactory.java @@ -0,0 +1,188 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Map; + +import javax.annotation.Nullable; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.schema.CompactionParams; + +/** + * The factory for compaction strategies and their containers. + */ +public class CompactionStrategyFactory +{ + private final ColumnFamilyStore cfs; + private final CompactionLogger compactionLogger; + + public CompactionStrategyFactory(ColumnFamilyStore cfs) + { + this.cfs = cfs; + this.compactionLogger = new CompactionLogger(cfs.metadata()); + } + + /** + * Reload the existing strategy container, possibly creating a new one if required. + * + * @param current the current strategy container, or {@code null} if this is the first time we're loading a + * compaction strategy + * @param compactionParams the new compaction parameters + * @param reason the reason for reloading + * + * @return Either a new strategy container or the current one, but reloaded with the given compaction parameters. + */ + public CompactionStrategyContainer reload(@Nullable CompactionStrategyContainer current, + CompactionParams compactionParams, + CompactionStrategyContainer.ReloadReason reason) + { + // If we were called due to a metadata change but the compaction parameters are the same then + // don't reload since we risk overriding parameters set via JMX + if (current != null && !current.shouldReload(compactionParams, reason)) + return current; + + Class containerClass = containerForStrategy(compactionParams.klass()); + CompactionStrategyContainer ret; + + // if the strategy belongs to the same container, we can just reload + if (current != null && current.getClass().equals(containerClass)) + ret = current.reload(current, compactionParams, reason); + else + { + // otherwise we need to re-create the container + ret = createStrategyContainer(containerClass, current, compactionParams, reason); + } + + if (ret != current) + cfs.getTracker().subscribe(ret); + + return ret; + } + + static boolean enableCompactionOnReload(@Nullable CompactionStrategyContainer previous, + CompactionParams compactionParams, + CompactionStrategyContainer.ReloadReason reason) + { + // If this is a JMX request, we only consider the params passed by it + if (reason == CompactionStrategyContainer.ReloadReason.JMX_REQUEST) + return compactionParams.isEnabled(); + // If the enabled state flag and the params of the previous container differ, compaction was forcefully + // enabled/disabled by JMX/nodetool, and we should inherit that setting through the enabled state flag + if (previous != null && previous.isEnabled() != previous.getCompactionParams().isEnabled()) + return previous.isEnabled(); + + return compactionParams.isEnabled(); + } + + /** + * Returns a {@link CompactionStrategyContainer#} class for the given strategy class. + * + * We need this method to create correct container for the strategy, but also to distinguish + * between situations when a container should reloaded or recreated. + */ + private Class containerForStrategy(Class strategyClass) + { + Class containerClass; + try + { + Field containerClassField = strategyClass.getField("CONTAINER_CLASS"); + containerClass = (Class) containerClassField.get(null); + } + catch (IllegalAccessException | NoSuchFieldException e) + { + containerClass = CompactionStrategyManager.class; + } + + return containerClass; + } + + private CompactionStrategyContainer createStrategyContainer(Class containerClass, + CompactionStrategyContainer previous, + CompactionParams compactionParams, + CompactionStrategyContainer.ReloadReason reason) + { + CompactionStrategyContainer ret; + try + { + Method createMethod = containerClass.getMethod("create", + CompactionStrategyContainer.class, + CompactionStrategyFactory.class, + CompactionParams.class, + CompactionStrategyContainer.ReloadReason.class); + ret = (CompactionStrategyContainer) createMethod.invoke(null, + previous, + this, + compactionParams, + reason); + } + catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) + { + ret = new CompactionStrategyManager(this); + ret.reload(previous, compactionParams, reason); + } + return ret; + } + + public CompactionLogger getCompactionLogger() + { + return compactionLogger; + } + + ColumnFamilyStore getCfs() + { + return cfs; + } + + /** + * Creates a compaction strategy that is managed by {@link CompactionStrategyManager} and its strategy holders. + * These strategies must extend {@link LegacyAbstractCompactionStrategy}. + * + * @return an instance of the compaction strategy specified in the parameters so long as it extends {@link LegacyAbstractCompactionStrategy} + * @throws IllegalArgumentException if the params do not contain a strategy that extends {@link LegacyAbstractCompactionStrategy} + */ + LegacyAbstractCompactionStrategy createLegacyStrategy(CompactionParams compactionParams) + { + try + { + if (!LegacyAbstractCompactionStrategy.class.isAssignableFrom(compactionParams.klass())) + throw new IllegalArgumentException("Expected compaction params for legacy strategy: " + compactionParams); + + Constructor constructor = + compactionParams.klass().getConstructor(CompactionStrategyFactory.class, Map.class); + LegacyAbstractCompactionStrategy ret = (LegacyAbstractCompactionStrategy) constructor.newInstance(this, compactionParams.options()); + compactionLogger.strategyCreated(ret); + return ret; + } + catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException e) + { + throw org.apache.cassandra.utils.Throwables.cleaned(e); + } + } + + /** + * Create a compaction strategy. This is only called by tiered storage so we forward to the legacy strategy. + */ + public CompactionStrategy createStrategy(CompactionParams compactionParams) + { + return createLegacyStrategy(compactionParams); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java index 129ee797ee0e..e1a88798989d 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyHolder.java @@ -29,7 +29,6 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.index.Index; @@ -43,25 +42,25 @@ public class CompactionStrategyHolder extends AbstractStrategyHolder { - private final List strategies = new ArrayList<>(); + private final List strategies = new ArrayList<>(); private final boolean isRepaired; - public CompactionStrategyHolder(ColumnFamilyStore cfs, DestinationRouter router, boolean isRepaired) + public CompactionStrategyHolder(ColumnFamilyStore cfs, CompactionStrategyFactory strategyFactory, DestinationRouter router, boolean isRepaired) { - super(cfs, router); + super(cfs, strategyFactory, router); this.isRepaired = isRepaired; } @Override public void startup() { - strategies.forEach(AbstractCompactionStrategy::startup); + strategies.forEach(CompactionStrategy::startup); } @Override public void shutdown() { - strategies.forEach(AbstractCompactionStrategy::shutdown); + strategies.forEach(CompactionStrategy::shutdown); } @Override @@ -69,7 +68,7 @@ public void setStrategyInternal(CompactionParams params, int numTokenPartitions) { strategies.clear(); for (int i = 0; i < numTokenPartitions; i++) - strategies.add(cfs.createCompactionStrategyInstance(params)); + strategies.add(strategyFactory.createLegacyStrategy(params)); } @Override @@ -89,24 +88,24 @@ public boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair, } @Override - public AbstractCompactionStrategy getStrategyFor(SSTableReader sstable) + public LegacyAbstractCompactionStrategy getStrategyFor(SSTableReader sstable) { Preconditions.checkArgument(managesSSTable(sstable), "Attempting to get compaction strategy from wrong holder"); return strategies.get(router.getIndexForSSTable(sstable)); } @Override - public Iterable allStrategies() + public Iterable allStrategies() { return strategies; } @Override - public Collection getBackgroundTaskSuppliers(int gcBefore) + public Collection getBackgroundTaskSuppliers(int gcBefore) { - List suppliers = new ArrayList<>(strategies.size()); - for (AbstractCompactionStrategy strategy : strategies) - suppliers.add(new TaskSupplier(strategy.getEstimatedRemainingTasks(), () -> strategy.getNextBackgroundTask(gcBefore))); + List suppliers = new ArrayList<>(strategies.size()); + for (CompactionStrategy strategy : strategies) + suppliers.add(new TasksSupplier(strategy.getEstimatedRemainingTasks(), () -> strategy.getNextBackgroundTasks(gcBefore))); return suppliers; } @@ -115,11 +114,9 @@ public Collection getBackgroundTaskSuppliers(int gcBefore) public Collection getMaximalTasks(int gcBefore, boolean splitOutput) { List tasks = new ArrayList<>(strategies.size()); - for (AbstractCompactionStrategy strategy : strategies) + for (CompactionStrategy strategy : strategies) { - Collection task = strategy.getMaximalTask(gcBefore, splitOutput); - if (task != null) - tasks.addAll(task); + tasks.addAll(strategy.getMaximalTasks(gcBefore, splitOutput)); } return tasks; } @@ -133,7 +130,7 @@ public Collection getUserDefinedTasks(GroupedSSTableCont if (sstables.isGroupEmpty(i)) continue; - tasks.add(strategies.get(i).getUserDefinedTask(sstables.getGroup(i), gcBefore)); + tasks.addAll(strategies.get(i).getUserDefinedTasks(sstables.getGroup(i), gcBefore)); } return tasks; } @@ -223,7 +220,7 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, boolean isTransient, MetadataCollector collector, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { if (isRepaired) @@ -239,7 +236,7 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, Preconditions.checkArgument(pendingRepair == null, "CompactionStrategyHolder can't create sstable writer with pendingRepair id"); // to avoid creating a compaction strategy for the wrong pending repair manager, we get the index based on where the sstable is to be written - AbstractCompactionStrategy strategy = strategies.get(router.getIndexForSSTableDirectory(descriptor)); + CompactionStrategy strategy = strategies.get(router.getIndexForSSTableDirectory(descriptor)); return strategy.createSSTableMultiWriter(descriptor, keyCount, repairedAt, @@ -247,16 +244,10 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, isTransient, collector, header, - indexes, + indexGroups, lifecycleNewTracker); } - @Override - public int getStrategyIndex(AbstractCompactionStrategy strategy) - { - return strategies.indexOf(strategy); - } - @Override public boolean containsSSTable(SSTableReader sstable) { diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java index deece30d45d9..e91224842de3 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java @@ -18,8 +18,6 @@ package org.apache.cassandra.db.compaction; -import java.io.File; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -27,52 +25,46 @@ import java.util.ConcurrentModificationException; import java.util.HashSet; import java.util.List; -import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Supplier; import java.util.stream.Collectors; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.primitives.Longs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.DiskBoundaries; import org.apache.cassandra.db.SerializationHeader; -import org.apache.cassandra.db.compaction.AbstractStrategyHolder.TaskSupplier; -import org.apache.cassandra.db.compaction.PendingRepairManager.CleanupTask; +import org.apache.cassandra.db.compaction.AbstractStrategyHolder.TasksSupplier; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.index.Index; -import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.ScannerList; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.io.sstable.metadata.StatsMetadata; import org.apache.cassandra.notifications.INotification; -import org.apache.cassandra.notifications.INotificationConsumer; import org.apache.cassandra.notifications.SSTableAddedNotification; import org.apache.cassandra.notifications.SSTableDeletingNotification; import org.apache.cassandra.notifications.SSTableListChangedNotification; import org.apache.cassandra.notifications.SSTableMetadataChanged; import org.apache.cassandra.notifications.SSTableRepairStatusChanged; -import org.apache.cassandra.repair.consistent.admin.CleanupSummary; import org.apache.cassandra.schema.CompactionParams; -import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ActiveRepairService; import static org.apache.cassandra.db.compaction.AbstractStrategyHolder.GroupedSSTableContainer; @@ -94,11 +86,11 @@ * * Whenever the {@link DiskBoundaries} change, the compaction strategies must be reloaded, so in order to ensure * the compaction strategy placement reflect most up-to-date disk boundaries, call {@link this#maybeReloadDiskBoundaries()} - * before acquiring the read lock to acess the strategies. + * before acquiring the read lock to access the strategies. * */ -public class CompactionStrategyManager implements INotificationConsumer +public class CompactionStrategyManager implements CompactionStrategyContainer { private static final Logger logger = LoggerFactory.getLogger(CompactionStrategyManager.class); public final CompactionLogger compactionLogger; @@ -129,25 +121,31 @@ public class CompactionStrategyManager implements INotificationConsumer private volatile boolean isActive = true; /* - We keep a copy of the schema compaction parameters here to be able to decide if we - should update the compaction strategy in maybeReload() due to an ALTER. + We keep a copy of the table metadata compaction parameters here to be able to decide if we + should update the compaction strategy due to a metadata change such as a schema changed + caused by an ALTER TABLE. - If a user changes the local compaction strategy and then later ALTERs a compaction parameter, - we will use the new compaction parameters. + If a user changes the local compaction strategy via JMX and then later ALTERs a compaction parameter, + we will use the new compaction parameters but we will not override the JMX parameters if compaction + was not changed by the ALTER. */ - private volatile CompactionParams schemaCompactionParams; + @SuppressWarnings("thread-safe") + private volatile CompactionParams metadataParams; private volatile boolean supportsEarlyOpen; private volatile int fanout; private volatile long maxSSTableSizeBytes; private volatile String name; - public CompactionStrategyManager(ColumnFamilyStore cfs) + public CompactionStrategyManager(CompactionStrategyFactory strategyFactory) { - this(cfs, cfs::getDiskBoundaries, cfs.getPartitioner().splitter().isPresent()); + this(strategyFactory, + () -> strategyFactory.getCfs().getDiskBoundaries(), + strategyFactory.getCfs().getPartitioner().splitter().isPresent()); } @VisibleForTesting - public CompactionStrategyManager(ColumnFamilyStore cfs, Supplier boundariesSupplier, + public CompactionStrategyManager(CompactionStrategyFactory strategyFactory, + Supplier boundariesSupplier, boolean partitionSSTablesByTokenRange) { AbstractStrategyHolder.DestinationRouter router = new AbstractStrategyHolder.DestinationRouter() @@ -162,65 +160,78 @@ public int getIndexForSSTableDirectory(Descriptor descriptor) return compactionStrategyIndexForDirectory(descriptor); } }; - transientRepairs = new PendingRepairHolder(cfs, router, true); - pendingRepairs = new PendingRepairHolder(cfs, router, false); - repaired = new CompactionStrategyHolder(cfs, router, true); - unrepaired = new CompactionStrategyHolder(cfs, router, false); + + cfs = strategyFactory.getCfs(); + + transientRepairs = new PendingRepairHolder(cfs, strategyFactory, router, true); + pendingRepairs = new PendingRepairHolder(cfs, strategyFactory, router, false); + repaired = new CompactionStrategyHolder(cfs, strategyFactory, router, true); + unrepaired = new CompactionStrategyHolder(cfs, strategyFactory, router, false); holders = ImmutableList.of(transientRepairs, pendingRepairs, repaired, unrepaired); - cfs.getTracker().subscribe(this); - logger.trace("{} subscribed to the data tracker.", this); - this.cfs = cfs; - this.compactionLogger = new CompactionLogger(cfs, this); + compactionLogger = strategyFactory.getCompactionLogger(); this.boundariesSupplier = boundariesSupplier; this.partitionSSTablesByTokenRange = partitionSSTablesByTokenRange; params = cfs.metadata().params.compaction; enabled = params.isEnabled(); - reload(cfs.metadata().params.compaction); + } + + public static CompactionStrategyContainer create(@Nullable CompactionStrategyContainer previous, + CompactionStrategyFactory strategyFactory, + CompactionParams compactionParams, + CompactionStrategyContainer.ReloadReason reason) + { + CompactionStrategyManager csm = new CompactionStrategyManager(strategyFactory); + csm.reload(previous != null ? previous : csm, compactionParams, reason); + return csm; } /** * Return the next background task * - * Returns a task for the compaction strategy that needs it the most (most estimated remaining tasks) - */ - public AbstractCompactionTask getNextBackgroundTask(int gcBefore) + * Legacy strategies will always return one task but we wrap this in a collection because new strategies + * might return multiple tasks. + * + * @return the task for the compaction strategy that needs it the most (most estimated remaining tasks) */ + @Override + public Collection getNextBackgroundTasks(int gcBefore) { maybeReloadDiskBoundaries(); readLock.lock(); try { if (!isEnabled()) - return null; + return ImmutableList.of(); int numPartitions = getNumTokenPartitions(); // first try to promote/demote sstables from completed repairs - AbstractCompactionTask repairFinishedTask; - repairFinishedTask = pendingRepairs.getNextRepairFinishedTask(); - if (repairFinishedTask != null) - return repairFinishedTask; + Collection repairFinishedTasks; + repairFinishedTasks = pendingRepairs.getNextRepairFinishedTasks(); + if (!repairFinishedTasks.isEmpty()) + return repairFinishedTasks; - repairFinishedTask = transientRepairs.getNextRepairFinishedTask(); - if (repairFinishedTask != null) - return repairFinishedTask; + repairFinishedTasks = transientRepairs.getNextRepairFinishedTasks(); + if (!repairFinishedTasks.isEmpty()) + return repairFinishedTasks; // sort compaction task suppliers by remaining tasks descending - List suppliers = new ArrayList<>(numPartitions * holders.size()); + List suppliers = new ArrayList<>(numPartitions * holders.size()); for (AbstractStrategyHolder holder : holders) suppliers.addAll(holder.getBackgroundTaskSuppliers(gcBefore)); Collections.sort(suppliers); - // return the first non-null task - for (TaskSupplier supplier : suppliers) + // return the first non-empty list, we could enhance it to return all tasks of all + // suppliers but this would change existing behavior + for (TasksSupplier supplier : suppliers) { - AbstractCompactionTask task = supplier.getTask(); - if (task != null) - return task; + Collection tasks = supplier.getTasks(); + if (!tasks.isEmpty()) + return tasks; } - return null; + return ImmutableList.of(); } finally { @@ -228,47 +239,25 @@ public AbstractCompactionTask getNextBackgroundTask(int gcBefore) } } - /** - * finds the oldest (by modification date) non-latest-version sstable on disk and creates an upgrade task for it - * @return - */ - @VisibleForTesting - @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute - AbstractCompactionTask findUpgradeSSTableTask() - { - if (!isEnabled() || !DatabaseDescriptor.automaticSSTableUpgrade()) - return null; - Set compacting = cfs.getTracker().getCompacting(); - List potentialUpgrade = cfs.getLiveSSTables() - .stream() - .filter(s -> !compacting.contains(s) && !s.descriptor.version.isLatestVersion()) - .sorted((o1, o2) -> { - File f1 = new File(o1.descriptor.filenameFor(Component.DATA)); - File f2 = new File(o2.descriptor.filenameFor(Component.DATA)); - return Longs.compare(f1.lastModified(), f2.lastModified()); - }).collect(Collectors.toList()); - for (SSTableReader sstable : potentialUpgrade) - { - LifecycleTransaction txn = cfs.getTracker().tryModify(sstable, OperationType.UPGRADE_SSTABLES); - if (txn != null) - { - logger.debug("Running automatic sstable upgrade for {}", sstable); - return getCompactionStrategyFor(sstable).getCompactionTask(txn, Integer.MIN_VALUE, Long.MAX_VALUE); - } - } - return null; + @Override + public CompactionLogger getCompactionLogger() + { + return compactionLogger; } + @Override public boolean isEnabled() { return enabled && isActive; } + @Override public boolean isActive() { return isActive; } + @Override public void resume() { writeLock.lock(); @@ -287,6 +276,7 @@ public void resume() * * Separate call from enable/disable to not have to save the enabled-state externally */ + @Override public void pause() { writeLock.lock(); @@ -301,7 +291,8 @@ public void pause() } - private void startup() + @Override + public void startup() { writeLock.lock(); try @@ -313,7 +304,6 @@ private void startup() } holders.forEach(AbstractStrategyHolder::startup); supportsEarlyOpen = repaired.first().supportsEarlyOpen(); - fanout = (repaired.first() instanceof LeveledCompactionStrategy) ? ((LeveledCompactionStrategy) repaired.first()).getLevelFanoutSize() : LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE; maxSSTableSizeBytes = repaired.first().getMaxSSTableBytes(); name = repaired.first().getName(); } @@ -322,25 +312,23 @@ private void startup() writeLock.unlock(); } - if (repaired.first().logAll) + if (repaired.first().getOptions().isLogAll()) compactionLogger.enable(); } /** - * return the compaction strategy for the given sstable - * * returns differently based on the repaired status and which vnode the compaction strategy belongs to * @param sstable - * @return + * @return the compaction strategy for the given sstable */ - public AbstractCompactionStrategy getCompactionStrategyFor(SSTableReader sstable) + LegacyAbstractCompactionStrategy getCompactionStrategyFor(SSTableReader sstable) { maybeReloadDiskBoundaries(); return compactionStrategyFor(sstable); } @VisibleForTesting - AbstractCompactionStrategy compactionStrategyFor(SSTableReader sstable) + LegacyAbstractCompactionStrategy compactionStrategyFor(SSTableReader sstable) { // should not call maybeReloadDiskBoundaries because it may be called from within lock readLock.lock(); @@ -376,7 +364,7 @@ int compactionStrategyIndexFor(SSTableReader sstable) if (!partitionSSTablesByTokenRange) return 0; - return currentBoundaries.getDiskIndex(sstable); + return currentBoundaries.getDiskIndexFromKey(sstable); } finally { @@ -421,19 +409,7 @@ PendingRepairHolder getTransientRepairsUnsafe() return transientRepairs; } - public boolean hasDataForPendingRepair(UUID sessionID) - { - readLock.lock(); - try - { - return pendingRepairs.hasDataForSession(sessionID) || transientRepairs.hasDataForSession(sessionID); - } - finally - { - readLock.unlock(); - } - } - + @Override public void shutdown() { writeLock.lock(); @@ -449,26 +425,6 @@ public void shutdown() } } - public void maybeReload(TableMetadata metadata) - { - // compare the old schema configuration to the new one, ignore any locally set changes. - if (metadata.params.compaction.equals(schemaCompactionParams)) - return; - - writeLock.lock(); - try - { - // compare the old schema configuration to the new one, ignore any locally set changes. - if (metadata.params.compaction.equals(schemaCompactionParams)) - return; - reload(metadata.params.compaction); - } - finally - { - writeLock.unlock(); - } - } - /** * Checks if the disk boundaries changed and reloads the compaction strategies * to reflect the most up-to-date disk boundaries. @@ -492,7 +448,7 @@ protected void maybeReloadDiskBoundaries() { if (!currentBoundaries.isOutOfDate()) return; - reload(params); + doReload(this, params, ReloadReason.DISK_BOUNDARIES_UPDATED); } finally { @@ -500,39 +456,67 @@ protected void maybeReloadDiskBoundaries() } } - /** - * Reload the compaction strategies - * - * Called after changing configuration and at startup. - * @param newCompactionParams - */ - private void reload(CompactionParams newCompactionParams) + @Override + public CompactionStrategyContainer reload(@Nonnull CompactionStrategyContainer previous, CompactionParams newCompactionParams, ReloadReason reason) { - boolean enabledWithJMX = enabled && !shouldBeEnabled(); - boolean disabledWithJMX = !enabled && shouldBeEnabled(); - - if (currentBoundaries != null) + writeLock.lock(); + try + { + doReload(previous, newCompactionParams, reason); + } + finally { - if (!newCompactionParams.equals(schemaCompactionParams)) - logger.debug("Recreating compaction strategy - compaction parameters changed for {}.{}", cfs.keyspace.getName(), cfs.getTableName()); - else if (currentBoundaries.isOutOfDate()) - logger.debug("Recreating compaction strategy - disk boundaries are out of date for {}.{}.", cfs.keyspace.getName(), cfs.getTableName()); + writeLock.unlock(); } + if (previous != this) + previous.shutdown(); + + return this; + } - if (currentBoundaries == null || currentBoundaries.isOutOfDate()) + private void doReload(CompactionStrategyContainer previous, CompactionParams compactionParams, ReloadReason reason) + { + boolean updateDiskBoundaries = currentBoundaries == null || currentBoundaries.isOutOfDate(); + boolean enabledOnReload = CompactionStrategyFactory.enableCompactionOnReload(previous, compactionParams, reason); + + logger.debug("Recreating compaction strategy for {}.{}, reason: {}, params updated: {}, disk boundaries updated: {}, enabled: {}, params: {} -> {}, metadataParams: {}", + cfs.getKeyspaceName(), cfs.getTableName(), reason, !compactionParams.equals(params), updateDiskBoundaries, enabledOnReload, params, compactionParams, metadataParams); + + if (updateDiskBoundaries) currentBoundaries = boundariesSupplier.get(); - setStrategy(newCompactionParams); - schemaCompactionParams = cfs.metadata().params.compaction; + int numPartitions = getNumTokenPartitions(); + for (AbstractStrategyHolder holder : holders) + holder.setStrategy(compactionParams, numPartitions); + + params = compactionParams; + + // full reload or switch from a strategy not managed by CompactionStrategyManager + if (metadataParams == null || reason == ReloadReason.FULL) + metadataParams = cfs.metadata().params.compaction; + else if (reason == ReloadReason.METADATA_CHANGE) + // metadataParams are aligned with compactionParams. We do not access TableParams.COMPACTION to avoid racing with + // concurrent ALTER TABLE metadata change. + metadataParams = compactionParams; + + // no-op for DISK_BOUNDARIES_UPDATED and JMX_REQUEST. DISK_BOUNDARIES_UPDATED does not change compaction params + // and JMX changes do not affect table metadata - if (disabledWithJMX || !shouldBeEnabled() && !enabledWithJMX) + + if (params.maxCompactionThreshold() <= 0 || params.minCompactionThreshold() <= 0) + { + logger.warn("Disabling compaction strategy by setting compaction thresholds to 0 is deprecated, set the compaction option 'enabled' to 'false' instead."); + disable(); + } + else if (!enabledOnReload) disable(); else enable(); + startup(); } - private Iterable getAllStrategies() + private Iterable getAllStrategies() { return Iterables.concat(Iterables.transform(holders, AbstractStrategyHolder::allStrategies)); } @@ -546,7 +530,7 @@ public int getUnleveledSSTables() if (repaired.first() instanceof LeveledCompactionStrategy) { int count = 0; - for (AbstractCompactionStrategy strategy : getAllStrategies()) + for (CompactionStrategy strategy : getAllStrategies()) count += ((LeveledCompactionStrategy) strategy).getLevelSize(0); return count; } @@ -558,11 +542,13 @@ public int getUnleveledSSTables() return 0; } + @Override public int getLevelFanoutSize() { - return fanout; + return repaired.first().getLevelFanoutSize(); } + @Override public int[] getSSTableCountPerLevel() { maybeReloadDiskBoundaries(); @@ -572,19 +558,22 @@ public int[] getSSTableCountPerLevel() if (repaired.first() instanceof LeveledCompactionStrategy) { int[] res = new int[LeveledGenerations.MAX_LEVEL_COUNT]; - for (AbstractCompactionStrategy strategy : getAllStrategies()) + for (CompactionStrategy strategy : getAllStrategies()) { int[] repairedCountPerLevel = ((LeveledCompactionStrategy) strategy).getAllLevelSize(); res = sumArrays(res, repairedCountPerLevel); } return res; } + else + { + return new int[0]; + } } finally { readLock.unlock(); } - return null; } static int[] sumArrays(int[] a, int[] b) @@ -724,7 +713,7 @@ private void handleRepairStatusChangedNotification(Iterable sstab */ private void handleMetadataChangedNotification(SSTableReader sstable, StatsMetadata oldMetadata) { - AbstractCompactionStrategy acs = getCompactionStrategyFor(sstable); + LegacyAbstractCompactionStrategy acs = getCompactionStrategyFor(sstable); acs.metadataChanged(oldMetadata, sstable); } @@ -775,6 +764,7 @@ else if (notification instanceof SSTableMetadataChanged) } } + @Override public void enable() { writeLock.lock(); @@ -789,6 +779,7 @@ public void enable() } } + @Override public void disable() { writeLock.lock(); @@ -811,7 +802,7 @@ public void disable() * @return */ @SuppressWarnings("resource") - public AbstractCompactionStrategy.ScannerList maybeGetScanners(Collection sstables, Collection> ranges) + private ScannerList maybeGetScanners(Collection sstables, Collection> ranges) { maybeReloadDiskBoundaries(); List scanners = new ArrayList<>(sstables.size()); @@ -835,10 +826,11 @@ public AbstractCompactionStrategy.ScannerList maybeGetScanners(Collection sstables, Collection> ranges) + @Override + public ScannerList getScanners(Collection sstables, Collection> ranges) { while (true) { @@ -853,11 +845,21 @@ public AbstractCompactionStrategy.ScannerList getScanners(Collection sstables) + @Override + public ScannerList getScanners(Collection sstables) { return getScanners(sstables, null); } + @Override + public Set getSSTables() + { + return getStrategies().stream() + .flatMap(strategy -> strategy.getSSTables().stream()) + .collect(Collectors.toSet()); + } + + @Override public Collection> groupSSTablesForAntiCompaction(Collection sstablesToGroup) { maybeReloadDiskBoundaries(); @@ -872,25 +874,26 @@ public Collection> groupSSTablesForAntiCompaction(Coll } } + @Override public long getMaxSSTableBytes() { return maxSSTableSizeBytes; } - public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes) + @Override + public AbstractCompactionTask createCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes) { maybeReloadDiskBoundaries(); readLock.lock(); try { validateForCompaction(txn.originals()); - return compactionStrategyFor(txn.originals().iterator().next()).getCompactionTask(txn, gcBefore, maxSSTableBytes); + return compactionStrategyFor(txn.originals().iterator().next()).createCompactionTask(txn, gcBefore, maxSSTableBytes); } finally { readLock.unlock(); } - } private void validateForCompaction(Iterable input) @@ -920,6 +923,7 @@ private void validateForCompaction(Iterable input) } } + @Override public CompactionTasks getMaximalTasks(final int gcBefore, final boolean splitOutput) { maybeReloadDiskBoundaries(); @@ -953,6 +957,7 @@ public CompactionTasks getMaximalTasks(final int gcBefore, final boolean splitOu * @param gcBefore gc grace period, throw away tombstones older than this * @return a list of compaction tasks corresponding to the sstables requested */ + @Override public CompactionTasks getUserDefinedTasks(Collection sstables, int gcBefore) { maybeReloadDiskBoundaries(); @@ -973,36 +978,41 @@ public CompactionTasks getUserDefinedTasks(Collection sstables, i } } + @Override public int getEstimatedRemainingTasks() { - maybeReloadDiskBoundaries(); - int tasks = 0; - readLock.lock(); - try - { - for (AbstractCompactionStrategy strategy : getAllStrategies()) - tasks += strategy.getEstimatedRemainingTasks(); - } - finally - { - readLock.unlock(); - } - return tasks; + return getStrategies(false).stream() + .flatMap(list -> list.stream()) + .mapToInt(CompactionStrategy::getEstimatedRemainingTasks) + .sum(); } - public boolean shouldBeEnabled() + @Override + public int getTotalCompactions() { - return params.isEnabled(); + return getStrategies(false).stream() + .flatMap(list -> list.stream()) + .mapToInt(CompactionStrategy::getTotalCompactions) + .sum(); } + @Override public String getName() { return name; } - public List> getStrategies() + @Override + public List getStrategies() { - maybeReloadDiskBoundaries(); + return getStrategies(true).stream().flatMap(List::stream).collect(Collectors.toList()); + } + + private List> getStrategies(boolean checkBoundaries) + { + if (checkBoundaries) + maybeReloadDiskBoundaries(); + readLock.lock(); try { @@ -1016,48 +1026,57 @@ public List> getStrategies() } } - public void setNewLocalCompactionStrategy(CompactionParams params) + @Override + public List getStrategies(boolean isRepaired, @Nullable UUID pendingRepair) { - logger.info("Switching local compaction strategy from {} to {}}", this.params, params); - writeLock.lock(); + readLock.lock(); try { - setStrategy(params); - if (shouldBeEnabled()) - enable(); + if (isRepaired) + return Lists.newArrayList(repaired.allStrategies()); + else if (pendingRepair != null) + return Lists.newArrayList(pendingRepairs.getStrategiesFor(pendingRepair)); else - disable(); - startup(); + return Lists.newArrayList(unrepaired.allStrategies()); } finally { - writeLock.unlock(); + readLock.unlock(); } } - private int getNumTokenPartitions() + /** + * @return the statistics for the compaction strategies that have compactions in progress or pending + */ + @Override + public List getStatistics() { - return partitionSSTablesByTokenRange ? currentBoundaries.directories.size() : 1; + return getStrategies(false).stream() + .flatMap(list -> list.stream()) + .filter(strategy -> strategy.getTotalCompactions() > 0) + .map(CompactionStrategy::getStatistics) + .flatMap(List::stream) + .collect(Collectors.toList()); } - private void setStrategy(CompactionParams params) + private int getNumTokenPartitions() { - int numPartitions = getNumTokenPartitions(); - for (AbstractStrategyHolder holder : holders) - holder.setStrategy(params, numPartitions); - this.params = params; + return partitionSSTablesByTokenRange && currentBoundaries != null ? currentBoundaries.directories.size() : 1; } + @Override public CompactionParams getCompactionParams() { return params; } - public boolean onlyPurgeRepairedTombstones() + @Override + public CompactionParams getMetadataCompactionParams() { - return Boolean.parseBoolean(params.options().get(AbstractCompactionStrategy.ONLY_PURGE_REPAIRED_TOMBSTONES)); + return metadataParams; } + @Override public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, @@ -1065,7 +1084,7 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, boolean isTransient, MetadataCollector collector, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { SSTable.validateRepairedMetadata(repairedAt, pendingRepair, isTransient); @@ -1080,7 +1099,7 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, isTransient, collector, header, - indexes, + indexGroups, lifecycleNewTracker); } finally @@ -1089,121 +1108,49 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, } } - public boolean isRepaired(AbstractCompactionStrategy strategy) - { - return repaired.getStrategyIndex(strategy) >= 0; - } - - public List getStrategyFolders(AbstractCompactionStrategy strategy) - { - readLock.lock(); - try - { - Directories.DataDirectory[] locations = cfs.getDirectories().getWriteableLocations(); - if (partitionSSTablesByTokenRange) - { - for (AbstractStrategyHolder holder : holders) - { - int idx = holder.getStrategyIndex(strategy); - if (idx >= 0) - return Collections.singletonList(locations[idx].location.getAbsolutePath()); - } - } - List folders = new ArrayList<>(locations.length); - for (Directories.DataDirectory location : locations) - { - folders.add(location.location.getAbsolutePath()); - } - return folders; - } - finally - { - readLock.unlock(); - } - } - + @Override public boolean supportsEarlyOpen() { return supportsEarlyOpen; } - @VisibleForTesting - List getPendingRepairManagers() + public ReentrantReadWriteLock.WriteLock getWriteLock() { - maybeReloadDiskBoundaries(); - readLock.lock(); - try - { - return Lists.newArrayList(pendingRepairs.getManagers()); - } - finally - { - readLock.unlock(); - } + return this.writeLock; } /** - * Mutates sstable repairedAt times and notifies listeners of the change with the writeLock held. Prevents races - * with other processes between when the metadata is changed and when sstables are moved between strategies. - */ - public void mutateRepaired(Collection sstables, long repairedAt, UUID pendingRepair, boolean isTransient) throws IOException + * This method is exposed for testing only + * @return the LocalSession sessionIDs of any pending repairs + */ + @VisibleForTesting + public Set pendingRepairs() { - Set changed = new HashSet<>(); - - writeLock.lock(); - try - { - for (SSTableReader sstable: sstables) - { - sstable.mutateRepairedAndReload(repairedAt, pendingRepair, isTransient); - verifyMetadata(sstable, repairedAt, pendingRepair, isTransient); - changed.add(sstable); - } - } - finally - { - try - { - // if there was an exception mutating repairedAt, we should still notify for the - // sstables that we were able to modify successfully before releasing the lock - cfs.getTracker().notifySSTableRepairedStatusChanged(changed); - } - finally - { - writeLock.unlock(); - } - } + Set ids = new HashSet<>(); + pendingRepairs.getManagers().forEach(p -> ids.addAll(p.getSessions())); + return ids; } - private static void verifyMetadata(SSTableReader sstable, long repairedAt, UUID pendingRepair, boolean isTransient) + @Override + public void repairSessionCompleted(UUID sessionID) { - if (!Objects.equals(pendingRepair, sstable.getPendingRepair())) - throw new IllegalStateException(String.format("Failed setting pending repair to %s on %s (pending repair is %s)", pendingRepair, sstable, sstable.getPendingRepair())); - if (repairedAt != sstable.getRepairedAt()) - throw new IllegalStateException(String.format("Failed setting repairedAt to %d on %s (repairedAt is %d)", repairedAt, sstable, sstable.getRepairedAt())); - if (isTransient != sstable.isTransient()) - throw new IllegalStateException(String.format("Failed setting isTransient to %b on %s (isTransient is %b)", isTransient, sstable, sstable.isTransient())); + for (PendingRepairManager manager : pendingRepairs.getManagers()) + manager.removeSessionIfEmpty(sessionID); } - public CleanupSummary releaseRepairData(Collection sessions) + // + // CompactionObserver - because the strategies observe compactions, for CSM this is currently a no-op + // + + @Override + public void onInProgress(CompactionProgress progress) { - List cleanupTasks = new ArrayList<>(); - readLock.lock(); - try - { - for (PendingRepairManager prm : Iterables.concat(pendingRepairs.getManagers(), transientRepairs.getManagers())) - cleanupTasks.add(prm.releaseSessionData(sessions)); - } - finally - { - readLock.unlock(); - } - CleanupSummary summary = new CleanupSummary(cfs, Collections.emptySet(), Collections.emptySet()); + } - for (CleanupTask task : cleanupTasks) - summary = CleanupSummary.add(summary, task.cleanup()); + @Override + public void onCompleted(UUID id) + { - return summary; } } diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyOptions.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyOptions.java new file mode 100644 index 000000000000..b4f3871edcf6 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyOptions.java @@ -0,0 +1,386 @@ +/* + * 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.compaction; + +import java.lang.reflect.InvocationTargetException; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import com.google.common.base.MoreObjects; + +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.utils.Throwables; + +import static java.lang.String.format; + +/** + * This class contains all compaction options that are shared by all strategies. + */ +public class CompactionStrategyOptions +{ + public static final int DEFAULT_MIN_THRESHOLD = 4; + public static final int DEFAULT_MAX_THRESHOLD = 32; + private static final Logger logger = LoggerFactory.getLogger(CompactionStrategyOptions.class); + + public static final Map DEFAULT_THRESHOLDS = + ImmutableMap.of(CompactionParams.Option.MIN_THRESHOLD.toString(), Integer.toString(DEFAULT_MIN_THRESHOLD), + CompactionParams.Option.MAX_THRESHOLD.toString(), Integer.toString(DEFAULT_MAX_THRESHOLD)); + + public static final String ONLY_PURGE_REPAIRED_TOMBSTONES = "only_purge_repaired_tombstones"; + + public static final String DEFAULT_TOMBSTONE_THRESHOLD = "0.2"; + // minimum interval needed to perform tombstone removal compaction in seconds, default 86400 or 1 day. + public static final String DEFAULT_TOMBSTONE_COMPACTION_INTERVAL = "86400"; + public static final String DEFAULT_UNCHECKED_TOMBSTONE_COMPACTION_OPTION = "false"; + public static final String DEFAULT_LOG_ALL_OPTION = "false"; + + public static final String TOMBSTONE_THRESHOLD_OPTION = "tombstone_threshold"; + public static final String TOMBSTONE_COMPACTION_INTERVAL_OPTION = "tombstone_compaction_interval"; + // disable range overlap check when deciding if an SSTable is candidate for tombstone compaction (CASSANDRA-6563) + public static final String UNCHECKED_TOMBSTONE_COMPACTION_OPTION = "unchecked_tombstone_compaction"; + public static final String LOG_ALL_OPTION = "log_all"; + public static final String COMPACTION_ENABLED = "enabled"; + + private final Class klass; + private final Map options; + private final float tombstoneThreshold; + private final long tombstoneCompactionInterval; + private final boolean uncheckedTombstoneCompaction; + private boolean disableTombstoneCompactions = false; + private final boolean logAll; + + public CompactionStrategyOptions(Class klass, Map options, boolean throwOnInvalidOption) + { + this.klass = klass; + this.options = copyOptions(klass, options); + + boolean useDefault = false; + try + { + validate(); // will throw ConfigurationException if the options are invalid + } + catch (ConfigurationException e) + { + // when called from CompactionParams we throw but when called from AbstractCompactionStrategy we use defaults + // could probably not bother with the latter (?) + if (throwOnInvalidOption) + { + throw e; + } + else + { + logger.warn("Error setting compaction strategy options ({}), defaults will be used", e.getMessage()); + useDefault = true; + } + } + + tombstoneThreshold = Float.parseFloat(getOption(TOMBSTONE_THRESHOLD_OPTION, useDefault, DEFAULT_TOMBSTONE_THRESHOLD)); + tombstoneCompactionInterval = Long.parseLong(getOption(TOMBSTONE_COMPACTION_INTERVAL_OPTION, useDefault, DEFAULT_TOMBSTONE_COMPACTION_INTERVAL)); + uncheckedTombstoneCompaction = Boolean.parseBoolean(getOption(UNCHECKED_TOMBSTONE_COMPACTION_OPTION, useDefault, DEFAULT_UNCHECKED_TOMBSTONE_COMPACTION_OPTION)); + logAll = Boolean.parseBoolean(getOption(LOG_ALL_OPTION, useDefault, DEFAULT_LOG_ALL_OPTION)); + } + + private Map copyOptions(Class klass, Map options) + { + Map newOptions = new HashMap<>(options); + + // For legacy compatibility reasons, for some compaction strategies we want to see the default min and max threshold + // in the compaction parameters that can be seen in CQL when retrieving the table from the schema tables so for + // these strategies we need to add these options when they have not been specified by the user + if (supportsThresholdParams(klass)) + { + newOptions.putIfAbsent(CompactionParams.Option.MIN_THRESHOLD.toString(), Integer.toString(DEFAULT_MIN_THRESHOLD)); + newOptions.putIfAbsent(CompactionParams.Option.MAX_THRESHOLD.toString(), Integer.toString(DEFAULT_MAX_THRESHOLD)); + } + + return newOptions; + } + + /** + * All strategies except {@link UnifiedCompactionStrategy} support the minimum and maximum thresholds + */ + @SuppressWarnings("unchecked") + public static boolean supportsThresholdParams(Class klass) + { + try + { + Map unrecognizedOptions = + (Map) klass.getMethod("validateOptions", Map.class) + .invoke(null, DEFAULT_THRESHOLDS); + + return unrecognizedOptions.isEmpty(); + } + catch (Exception e) + { + throw Throwables.cleaned(e); + } + } + + private String getOption(String optionName, boolean useDefault, String defaultValue) + { + if (useDefault) + return defaultValue; + + String optionValue = options.get(optionName); + if (optionValue == null) + return defaultValue; + + return optionValue; + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("class", klass.getName()) + .add("options", options) + .toString(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof CompactionStrategyOptions)) + return false; + + CompactionStrategyOptions that = (CompactionStrategyOptions) o; + + return klass.equals(that.klass) && options.equals(that.options); + } + + @Override + public int hashCode() + { + return Objects.hash(klass, options); + } + + private Map validate() + { + try + { + // Each strategy currently implements a static validateOptions() method for custom validation, the default behavior + // is to simply call validateOptions() below, through AbstractCompactionStrategy.validateOptions(), we could simplify + // all this assuming we don't need to support any user-defined compaction strategy + Map unknownOptions = (Map) klass.getMethod("validateOptions", Map.class).invoke(null, options); + if (!unknownOptions.isEmpty()) + { + throw new ConfigurationException(format("Properties specified %s are not understood by %s", + unknownOptions.keySet(), + klass.getSimpleName())); + } + + return unknownOptions; + } + catch (NoSuchMethodException e) + { + logger.warn("Compaction strategy {} does not have a static validateOptions method. Validation ignored", klass.getName()); + } + catch (InvocationTargetException e) + { + if (e.getTargetException() instanceof ConfigurationException) + throw (ConfigurationException) e.getTargetException(); + + Throwable cause = e.getCause() == null + ? e + : e.getCause(); + + throw new ConfigurationException(format("%s.validateOptions() threw an error: %s %s", + klass.getName(), + cause.getClass().getName(), + cause.getMessage()), + e); + } + catch (IllegalAccessException e) + { + throw new ConfigurationException("Cannot access method validateOptions in " + klass.getName(), e); + } + + if (minCompactionThreshold() <= 0 || maxCompactionThreshold() <= 0) + { + throw new ConfigurationException("Disabling compaction by setting compaction thresholds to 0 has been removed," + + " set the compaction option 'enabled' to false instead."); + } + + if (minCompactionThreshold() <= 1) + { + throw new ConfigurationException(format("Min compaction threshold cannot be less than 2 (got %d)", + minCompactionThreshold())); + } + + if (minCompactionThreshold() > maxCompactionThreshold()) + { + throw new ConfigurationException(format("Min compaction threshold (got %d) cannot be greater than max compaction threshold (got %d)", + minCompactionThreshold(), + maxCompactionThreshold())); + } + + return options; + } + + public static Map validateOptions(Map options) throws ConfigurationException + { + String minThreshold = options.get(CompactionParams.Option.MIN_THRESHOLD.toString()); + if (minThreshold != null && !StringUtils.isNumeric(minThreshold)) + { + throw new ConfigurationException(format("Invalid value %s for '%s' compaction sub-option - must be an integer", + minThreshold, + CompactionParams.Option.MIN_THRESHOLD)); + } + + String maxThreshold = options.get(CompactionParams.Option.MAX_THRESHOLD.toString()); + if (maxThreshold != null && !StringUtils.isNumeric(maxThreshold)) + { + throw new ConfigurationException(format("Invalid value %s for '%s' compaction sub-option - must be an integer", + maxThreshold, + CompactionParams.Option.MAX_THRESHOLD)); + } + + String threshold = options.get(TOMBSTONE_THRESHOLD_OPTION); + if (threshold != null) + { + try + { + float thresholdValue = Float.parseFloat(threshold); + if (thresholdValue < 0) + { + throw new ConfigurationException(String.format("%s must be greater than 0, but was %f", TOMBSTONE_THRESHOLD_OPTION, thresholdValue)); + } + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format("%s is not a parsable int (base10) for %s", threshold, TOMBSTONE_THRESHOLD_OPTION), e); + } + } + + String interval = options.get(TOMBSTONE_COMPACTION_INTERVAL_OPTION); + if (interval != null) + { + try + { + long tombstoneCompactionInterval = Long.parseLong(interval); + if (tombstoneCompactionInterval < 0) + { + throw new ConfigurationException(String.format("%s must be greater than 0, but was %d", TOMBSTONE_COMPACTION_INTERVAL_OPTION, tombstoneCompactionInterval)); + } + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format("%s is not a parsable int (base10) for %s", interval, TOMBSTONE_COMPACTION_INTERVAL_OPTION), e); + } + } + + String unchecked = options.get(UNCHECKED_TOMBSTONE_COMPACTION_OPTION); + if (unchecked != null && !unchecked.equalsIgnoreCase("true") && !unchecked.equalsIgnoreCase("false")) + { + throw new ConfigurationException(String.format("'%s' should be either 'true' or 'false', not '%s'", UNCHECKED_TOMBSTONE_COMPACTION_OPTION, unchecked)); + } + + String logAll = options.get(LOG_ALL_OPTION); + if (logAll != null && !logAll.equalsIgnoreCase("true") && !logAll.equalsIgnoreCase("false")) + { + throw new ConfigurationException(String.format("'%s' should either be 'true' or 'false', not %s", LOG_ALL_OPTION, logAll)); + } + + String compactionEnabled = options.get(COMPACTION_ENABLED); + if (compactionEnabled != null && !compactionEnabled.equalsIgnoreCase("true") && !compactionEnabled.equalsIgnoreCase("false")) + { + throw new ConfigurationException(String.format("enabled should either be 'true' or 'false', not %s", compactionEnabled)); + } + + Map uncheckedOptions = new HashMap<>(options); + uncheckedOptions.remove(TOMBSTONE_THRESHOLD_OPTION); + uncheckedOptions.remove(TOMBSTONE_COMPACTION_INTERVAL_OPTION); + uncheckedOptions.remove(UNCHECKED_TOMBSTONE_COMPACTION_OPTION); + uncheckedOptions.remove(LOG_ALL_OPTION); + uncheckedOptions.remove(COMPACTION_ENABLED); + uncheckedOptions.remove(ONLY_PURGE_REPAIRED_TOMBSTONES); + uncheckedOptions.remove(CompactionParams.Option.PROVIDE_OVERLAPPING_TOMBSTONES.toString()); + return uncheckedOptions; + } + + public int minCompactionThreshold() + { + String threshold = options.get(CompactionParams.Option.MIN_THRESHOLD.toString()); + return threshold == null + ? DEFAULT_MIN_THRESHOLD + : Integer.parseInt(threshold); + } + + public int maxCompactionThreshold() + { + String threshold = options.get(CompactionParams.Option.MAX_THRESHOLD.toString()); + return threshold == null + ? DEFAULT_MAX_THRESHOLD + : Integer.parseInt(threshold); + } + + public Class klass() + { + return klass; + } + + public Map getOptions() + { + return options; + } + + public float getTombstoneThreshold() + { + return tombstoneThreshold; + } + + public long getTombstoneCompactionInterval() + { + return tombstoneCompactionInterval; + } + + public boolean isUncheckedTombstoneCompaction() + { + return uncheckedTombstoneCompaction; + } + + public boolean isDisableTombstoneCompactions() + { + return disableTombstoneCompactions; + } + + /** + * {@link DateTieredCompactionStrategy} and {@link TimeWindowCompactionStrategy} disable this + * parameter if other parameters aren't available. + */ + public void setDisableTombstoneCompactions(boolean disableTombstoneCompactions) + { + this.disableTombstoneCompactions = disableTombstoneCompactions; + } + + public boolean isLogAll() + { + return logAll; + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyStatistics.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyStatistics.java new file mode 100644 index 000000000000..307810b912ee --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyStatistics.java @@ -0,0 +1,159 @@ +/* + * 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.compaction; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableList; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.cassandra.schema.TableMetadata; + +/** + * The statistics for a compaction strategy, to be published over JMX and insights. + *

    + * Implements serializable to allow structured info to be returned via JMX. The JSON + * properties are published to insights so changing them has a downstream impact. + */ +public class CompactionStrategyStatistics implements Serializable +{ + private static final long serialVersionUID = 3695927592357744816L; + + private final String keyspace; + private final String table; + private final String strategy; + private final List aggregates; + + CompactionStrategyStatistics(TableMetadata metadata, + String strategy, + List aggregates) + { + this.keyspace = metadata.keyspace; + this.table = metadata.name; + this.strategy = strategy; + this.aggregates = new ArrayList<>(aggregates); + } + + public String keyspace() + { + return keyspace; + } + + public String table() + { + return table; + } + + @JsonProperty + public String strategy() + { + return strategy; + } + + @JsonProperty + public List aggregates() + { + return aggregates; + } + + @Override + public String toString() + { + StringBuilder ret = new StringBuilder(1024); + ret.append(keyspace) + .append('.') + .append(table) + .append('/') + .append(strategy) + .append('\n'); + + if (!aggregates.isEmpty()) + { + Collection header = aggregates.get(0).header(); // all headers are identical + int[] lengths = new int[header.size()]; // the max lengths of each column + Iterator it = header.iterator(); + + for (int i = 0; i < lengths.length; i++) + lengths[i] = it.next().length(); + + Map> rowsByShard = new LinkedHashMap<>(); + for (CompactionAggregateStatistics aggregate : aggregates) + { + String shard = aggregate.shard(); + List rows = rowsByShard.computeIfAbsent(shard, key -> new ArrayList<>(aggregates.size())); + String[] data = new String[header.size()]; + + it = aggregate.data().iterator(); + for (int i = 0; i < lengths.length; i++) + { + data[i] = it.next(); + if (data[i].length() > lengths[i]) + lengths[i] = data[i].length(); + } + + rows.add(data); + } + + for (Map.Entry> entry : rowsByShard.entrySet()) + { + // optional shard + if (!entry.getKey().isEmpty()) + ret.append("Shard/").append(entry.getKey()).append('\n'); + + // header + it = header.iterator(); + for (int i = 0; i < header.size(); i++) + ret.append(String.format("%-" + lengths[i] + "s\t", it.next())); + + ret.append('\n'); + + // rows + for (String[] row : entry.getValue()) + { + for (int i = 0; i < row.length; i++) + ret.append(String.format("%-" + lengths[i] + "s\t", row[i])); + + ret.append('\n'); + } + + ret.append('\n'); + } + } + + return ret.toString(); + } + + Collection getHeader() + { + return aggregates.isEmpty() ? ImmutableList.of() : aggregates.get(0).header(); + } + + Collection> getData() + { + return aggregates.stream().map(CompactionAggregateStatistics::data).collect(Collectors.toList()); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java index 13c97253bcf6..51582fbf4615 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java @@ -17,15 +17,20 @@ */ package org.apache.cassandra.db.compaction; +import java.io.Closeable; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import javax.annotation.Nullable; -import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import com.google.common.util.concurrent.RateLimiter; @@ -40,45 +45,96 @@ import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.io.sstable.ScannerList; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.Refs; +import static org.apache.cassandra.db.compaction.CompactionManager.compactionRateLimiterAcquire; +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemoryPerSecond; + public class CompactionTask extends AbstractCompactionTask { protected static final Logger logger = LoggerFactory.getLogger(CompactionTask.class); + protected final int gcBefore; protected final boolean keepOriginals; - protected static long totalBytesCompacted = 0; - private ActiveCompactionsTracker activeCompactions; + /** for trace logging purposes only */ + private static final AtomicLong totalBytesCompacted = new AtomicLong(); + + // The compaction strategy is not necessarily available for all compaction tasks (e.g. GC or sstable splitting) + @Nullable + private final CompactionStrategy strategy; + + public CompactionTask(ColumnFamilyStore cfs, + LifecycleTransaction txn, + int gcBefore, + boolean keepOriginals, + @Nullable CompactionStrategy strategy) + { + super(cfs, txn); + this.gcBefore = gcBefore; + this.keepOriginals = keepOriginals; + this.strategy = strategy; - public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore) + if (strategy != null) + addObserver(strategy); + + logger.debug("Created compaction task with id {} and strategy {}", txn.opId(), strategy); + } + + /** + * Create a compaction task without a compaction strategy, currently only called by tests. + */ + static AbstractCompactionTask forTesting(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore) { - this(cfs, txn, gcBefore, false); + return new CompactionTask(cfs, txn, gcBefore, false, null); } - public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean keepOriginals) + /** + * Create a compaction task for deleted data collection. + */ + public static AbstractCompactionTask forGarbageCollection(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, CompactionParams.TombstoneOption tombstoneOption) { - super(cfs, txn); - this.gcBefore = gcBefore; - this.keepOriginals = keepOriginals; + AbstractCompactionTask task = new CompactionTask(cfs, txn, gcBefore, false, null) + { + @Override + protected CompactionController getCompactionController(Set toCompact) + { + return new CompactionController(cfs, toCompact, gcBefore, null, tombstoneOption); + } + + @Override + protected int getLevel() + { + return txn.onlyOne().getSSTableLevel(); + } + }; + task.setUserDefined(true); + task.setCompactionType(OperationType.GARBAGE_COLLECT); + return task; } - public static synchronized long addToTotalBytesCompacted(long bytesCompacted) + private static long addToTotalBytesCompacted(long bytesCompacted) { - return totalBytesCompacted += bytesCompacted; + return totalBytesCompacted.addAndGet(bytesCompacted); } - protected int executeInternal(ActiveCompactionsTracker activeCompactions) + @Override + protected int executeInternal() { - this.activeCompactions = activeCompactions == null ? ActiveCompactionsTracker.NOOP : activeCompactions; run(); return transaction.originals().size(); } - public boolean reduceScopeForLimitedSpace(Set nonExpiredSSTables, long expectedSize) + private boolean reduceScopeForLimitedSpace(Set nonExpiredSSTables, long expectedSize) { if (partialCompactionsAcceptable() && transaction.originals().size() > 1) { @@ -101,6 +157,7 @@ public boolean reduceScopeForLimitedSpace(Set nonExpiredSSTables, * which are properly serialized. * Caller is in charge of marking/unmarking the sstables as compacting. */ + @Override protected void runMayThrow() throws Exception { // The collection of sstables passed may be empty (but not null); even if @@ -110,162 +167,410 @@ protected void runMayThrow() throws Exception if (transaction.originals().isEmpty()) return; - // Note that the current compaction strategy, is not necessarily the one this task was created under. - // This should be harmless; see comments to CFS.maybeReloadCompactionStrategy. - CompactionStrategyManager strategy = cfs.getCompactionStrategyManager(); - if (DatabaseDescriptor.isSnapshotBeforeCompaction()) - cfs.snapshotWithoutFlush(System.currentTimeMillis() + "-compact-" + cfs.name); + cfs.snapshotWithoutMemtable(System.currentTimeMillis() + "-compact-" + cfs.name); + + try (CompactionController controller = getCompactionController(transaction.originals()); + CompactionOperation operation = new CompactionOperation(controller)) + { + operation.execute(); + } + } - try (CompactionController controller = getCompactionController(transaction.originals())) + /** + * The compaction operation is a special case of an {@link AbstractTableOperation} and takes care of executing the + * actual compaction and releasing any resources when the compaction is finished. + *

    + * This class also extends {@link AbstractTableOperation} for reporting compaction-specific progress information. + */ + public final class CompactionOperation implements AutoCloseable + { + private final CompactionController controller; + private final CompactionStrategy strategy; + private final Set fullyExpiredSSTables; + private final UUID taskId; + private final RateLimiter limiter; + private final long startNanos; + private final long startTime; + private final Set actuallyCompact; + private final CompactionProgress progress; + + // resources that are updated and may be read by another thread + private volatile Collection newSStables; + private volatile long totalKeysWritten; + private volatile long estimatedKeys; + + // resources that are updated but only read by this thread + private boolean completed; + + // resources that need closing + private Refs sstableRefs; + private ScannerList scanners; + private CompactionIterator compactionIterator; + private TableOperation op; + private Closeable obsCloseable; + private CompactionAwareWriter writer; + + /** + * Create a new compaction operation. + *

    + * @param controller the compaction controller is needed by the scanners and compaction iterator to manage options + */ + private CompactionOperation(CompactionController controller) { + this.controller = controller; - final Set fullyExpiredSSTables = controller.getFullyExpiredSSTables(); + this.strategy = cfs.getCompactionStrategy(); + this.fullyExpiredSSTables = controller.getFullyExpiredSSTables(); + this.taskId = transaction.opId(); // select SSTables to compact based on available disk space. buildCompactionCandidatesForAvailableDiskSpace(fullyExpiredSSTables); // sanity check: all sstables must belong to the same cfs - assert !Iterables.any(transaction.originals(), new Predicate() + assert !Iterables.any(transaction.originals(), sstable -> !sstable.descriptor.cfname.equals(cfs.name)); + + this.limiter = CompactionManager.instance.getRateLimiter(); + this.startNanos = System.nanoTime(); + this.startTime = System.currentTimeMillis(); + this.actuallyCompact = Sets.difference(transaction.originals(), fullyExpiredSSTables); + this.progress = new Progress(); + this.newSStables = Collections.emptyList(); + this.totalKeysWritten = 0; + this.estimatedKeys = 0; + this.completed = false; + + Directories dirs = getDirectories(); + + try { - @Override - public boolean apply(SSTableReader sstable) - { - return !sstable.descriptor.cfname.equals(cfs.name); - } - }); + // resources that need closing, must be created last in case of exceptions and released if there is an exception in the c.tor + this.sstableRefs = Refs.ref(actuallyCompact); + this.scanners = strategy.getScanners(actuallyCompact); + this.compactionIterator = new CompactionIterator(compactionType, scanners.scanners, controller, FBUtilities.nowInSeconds(), taskId); + this.op = compactionIterator.getOperation(); + this.writer = getCompactionAwareWriter(cfs, dirs, transaction, actuallyCompact); + this.obsCloseable = opObserver.onOperationStart(op); + + compObservers.forEach(obs -> obs.onInProgress(progress)); + } + catch (Throwable t) + { + t = Throwables.close(t, obsCloseable, writer, compactionIterator, scanners, sstableRefs); // ok to close even if null + + Throwables.maybeFail(t); + } + } - UUID taskId = transaction.opId(); + private void execute() + { + try + { + execute0(); + } + catch (Throwable t) + { + Throwables.maybeFail(onError(t)); + } + } + private void execute0() + { // new sstables from flush can be added during a compaction, but only the compaction can remove them, // so in our single-threaded compaction world this is a valid way of determining if we're compacting // all the sstables (that existed when we started) - StringBuilder ssTableLoggerMsg = new StringBuilder("["); - for (SSTableReader sstr : transaction.originals()) + if (logger.isDebugEnabled()) { - ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel())); + debugLogCompactingMessage(taskId); } - ssTableLoggerMsg.append("]"); - - logger.info("Compacting ({}) {}", taskId, ssTableLoggerMsg); - RateLimiter limiter = CompactionManager.instance.getRateLimiter(); - long start = System.nanoTime(); - long startTime = System.currentTimeMillis(); - long totalKeysWritten = 0; - long estimatedKeys = 0; - long inputSizeBytes; + long lastCheckObsoletion = startNanos; + double compressionRatio = scanners.getCompressionRatio(); + if (compressionRatio == MetadataCollector.NO_COMPRESSION_RATIO) + compressionRatio = 1.0; - Set actuallyCompact = Sets.difference(transaction.originals(), fullyExpiredSSTables); - Collection newSStables; + long lastBytesScanned = 0; - long[] mergedRowCounts; - long totalSourceCQLRows; + if (!controller.cfs.getCompactionStrategyContainer().isActive()) + throw new CompactionInterruptedException(op.getProgress()); - // SSTableScanners need to be closed before markCompactedSSTablesReplaced call as scanners contain references - // to both ifile and dfile and SSTR will throw deletion errors on Windows if it tries to delete before scanner is closed. - // See CASSANDRA-8019 and CASSANDRA-8399 - int nowInSec = FBUtilities.nowInSeconds(); - try (Refs refs = Refs.ref(actuallyCompact); - AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact); - CompactionIterator ci = new CompactionIterator(compactionType, scanners.scanners, controller, nowInSec, taskId)) + estimatedKeys = writer.estimatedKeys(); + while (compactionIterator.hasNext()) { - long lastCheckObsoletion = start; - inputSizeBytes = scanners.getTotalCompressedSize(); - double compressionRatio = scanners.getCompressionRatio(); - if (compressionRatio == MetadataCollector.NO_COMPRESSION_RATIO) - compressionRatio = 1.0; + if (op.isStopRequested()) + throw new CompactionInterruptedException(op.getProgress()); - long lastBytesScanned = 0; + UnfilteredRowIterator partition = compactionIterator.next(); + if (writer.append(partition)) + totalKeysWritten++; - activeCompactions.beginCompaction(ci); - try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, getDirectories(), transaction, actuallyCompact)) - { - // Note that we need to re-check this flag after calling beginCompaction above to avoid a window - // where the compaction does not exist in activeCompactions but the CSM gets paused. - // We already have the sstables marked compacting here so CompactionManager#waitForCessation will - // block until the below exception is thrown and the transaction is cancelled. - if (!controller.cfs.getCompactionStrategyManager().isActive()) - throw new CompactionInterruptedException(ci.getCompactionInfo()); - estimatedKeys = writer.estimatedKeys(); - while (ci.hasNext()) - { - if (writer.append(ci.next())) - totalKeysWritten++; - - - long bytesScanned = scanners.getTotalBytesScanned(); - - //Rate limit the scanners, and account for compression - CompactionManager.compactionRateLimiterAcquire(limiter, bytesScanned, lastBytesScanned, compressionRatio); - - lastBytesScanned = bytesScanned; - - if (System.nanoTime() - lastCheckObsoletion > TimeUnit.MINUTES.toNanos(1L)) - { - controller.maybeRefreshOverlaps(); - lastCheckObsoletion = System.nanoTime(); - } - } - - // point of no return - newSStables = writer.finish(); - } - finally + long bytesScanned = scanners.getTotalBytesScanned(); + + // Rate limit the scanners, and account for compression + if (compactionRateLimiterAcquire(limiter, bytesScanned, lastBytesScanned, compressionRatio)) + lastBytesScanned = bytesScanned; + + long now = System.nanoTime(); + if (now - lastCheckObsoletion > TimeUnit.MINUTES.toNanos(1L)) { - activeCompactions.finishCompaction(ci); - mergedRowCounts = ci.getMergedRowCounts(); - totalSourceCQLRows = ci.getTotalSourceCQLRows(); + controller.maybeRefreshOverlaps(); + lastCheckObsoletion = now; } } + // point of no return + newSStables = writer.finish(); + + + completed = true; + } + + private Throwable onError(Throwable e) + { + if (e instanceof AssertionError) + { + // Add additional information to help operators. + AssertionError error = new AssertionError( + String.format("Illegal input has been generated, most probably due to corruption in the input sstables\n" + + "\t%s\n" + + "Try scrubbing the sstables by running\n" + + "\tnodetool scrub %s %s\n", + transaction.originals(), + cfs.keyspace.getName(), + cfs.getTableName())); + error.addSuppressed(e); + return error; + } + + return e; + } + + // + // Closeable + // + + @Override + public void close() + { + Throwable err = Throwables.close((Throwable) null, obsCloseable, writer, compactionIterator, scanners, sstableRefs); + if (transaction.isOffline()) { - Refs.release(Refs.selfRefs(newSStables)); - } - else - { - // log a bunch of statistics about the result and save to system table compaction_history - - long durationInNano = System.nanoTime() - start; - long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano); - long startsize = inputSizeBytes; - long endsize = SSTableReader.getTotalBytes(newSStables); - double ratio = (double) endsize / (double) startsize; - - StringBuilder newSSTableNames = new StringBuilder(); - for (SSTableReader reader : newSStables) - newSSTableNames.append(reader.descriptor.baseFilename()).append(","); - long totalSourceRows = 0; - for (int i = 0; i < mergedRowCounts.length; i++) - totalSourceRows += mergedRowCounts[i] * (i + 1); - - String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getTableName(), mergedRowCounts, startsize, endsize); - - logger.info(String.format("Compacted (%s) %d sstables to [%s] to level=%d. %s to %s (~%d%% of original) in %,dms. Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s. %,d total partitions merged to %,d. Partition merge counts were {%s}", - taskId, - transaction.originals().size(), - newSSTableNames.toString(), - getLevel(), - FBUtilities.prettyPrintMemory(startsize), - FBUtilities.prettyPrintMemory(endsize), - (int) (ratio * 100), - dTime, - FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano), - FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano), - (int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1), - totalSourceRows, - totalKeysWritten, - mergeSummary)); + Refs.release(Refs.selfRefs(newSStables)); // this is harmless in case of exception, newSStables will be empty + } + else if (completed) + { + // This code used to execute only if the compaction was successful so we preserve the existing behavior + updateCompactionHistory(taskId, cfs.keyspace.getName(), cfs.getTableName(), progress); + CompactionManager.instance.incrementRemovedExpiredSSTables(fullyExpiredSSTables.size()); + if (transaction.originals().size() > 0 && actuallyCompact.size() == 0) + // this CompactionOperation only deleted fully expired SSTables without compacting anything + CompactionManager.instance.incrementDeleteOnlyCompactions(); + + if (logger.isDebugEnabled()) + { + debugLogCompactionSummaryInfo(taskId, System.nanoTime() - startNanos, totalKeysWritten, newSStables, progress); + } if (logger.isTraceEnabled()) { - logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize))); - logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten)); + traceLogCompactionSummaryInfo(totalKeysWritten, estimatedKeys, progress); } - cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables); + strategy.getCompactionLogger().compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables); // update the metrics - cfs.metric.compactionBytesWritten.inc(endsize); + cfs.metric.incBytesCompacted(progress.adjustedInputDiskSize(), + progress.outputDiskSize(), + System.nanoTime() - startNanos); + } + + Throwables.maybeFail(err); + } + + // + // CompactionProgress + // + + private final class Progress implements CompactionProgress + { + // + // TableOperation.Progress methods + // + + @Override + public Optional keyspace() + { + return Optional.of(metadata().keyspace); + } + + @Override + public Optional table() + { + return Optional.of(metadata().name); + } + + @Override + public TableMetadata metadata() + { + return cfs.metadata(); + } + + /** + * @return the number of bytes read by the compaction iterator. For compressed or encrypted sstables, + * this is the number of bytes processed by the iterator after decompression, so this is the current + * position in the uncompressed sstable files. + */ + @Override + public long completed() + { + return compactionIterator.bytesRead(); + } + + /** + * @return the initial number of bytes for input sstables. For compressed or encrypted sstables, + * this is the number of bytes after decompression, so this is the uncompressed length of sstable files. + */ + public long total() + { + return compactionIterator.totalBytes(); + } + + @Override + public OperationType operationType() + { + return compactionType; + } + + @Override + public UUID operationId() + { + return taskId; + } + + @Override + public TableOperation.Unit unit() + { + return TableOperation.Unit.BYTES; + } + + @Override + public Set sstables() + { + return transaction.originals(); + } + + // + // CompactionProgress + // + + @Override + @Nullable + public CompactionStrategy strategy() + { + return CompactionTask.this.strategy; + } + + @Override + public boolean isStopRequested() + { + return op.isStopRequested(); + } + + @Override + public Collection inSSTables() + { + // TODO should we use transaction.originals() and include the expired sstables? + // This would be more correct but all the metrics we get from CompactionIterator will not be compatible + return actuallyCompact; + } + + @Override + public Collection outSSTables() + { + return newSStables; + } + + @Override + public long inputDiskSize() + { + return SSTableReader.getTotalBytes(actuallyCompact); + } + + @Override + public long inputUncompressedSize() + { + return compactionIterator.totalBytes(); + } + + @Override + public long adjustedInputDiskSize() + { + return scanners.getTotalCompressedSize(); + } + + @Override + public long outputDiskSize() + { + return SSTableReader.getTotalBytes(newSStables); + } + + @Override + public long uncompressedBytesRead() + { + return compactionIterator.bytesRead(); + } + + @Override + public long uncompressedBytesRead(int level) + { + return compactionIterator.bytesRead(level); + } + + @Override + public long uncompressedBytesWritten() + { + return writer.bytesWritten(); + } + + @Override + public long durationInNanos() + { + return System.nanoTime() - startNanos; + } + + @Override + public long partitionsRead() + { + return compactionIterator.totalSourcePartitions(); + } + + @Override + public long rowsRead() + { + return compactionIterator.totalSourceRows(); + } + + @Override + public long[] partitionsHistogram() + { + return compactionIterator.mergedPartitionsHistogram(); + } + + @Override + public long[] rowsHistogram() + { + return compactionIterator.mergedRowsHistogram(); + } + + @Override + public double sizeRatio() + { + long estInputSizeBytes = adjustedInputDiskSize(); + if (estInputSizeBytes > 0) + return outputDiskSize() / (double) estInputSizeBytes; + + // this is a valid case, when there are no sstables to actually compact + // the previous code would return a NaN that would be logged as zero + return 0; } } } @@ -279,24 +584,6 @@ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, return new DefaultCompactionWriter(cfs, directories, transaction, nonExpiredSSTables, keepOriginals, getLevel()); } - public static String updateCompactionHistory(String keyspaceName, String columnFamilyName, long[] mergedRowCounts, long startSize, long endSize) - { - StringBuilder mergeSummary = new StringBuilder(mergedRowCounts.length * 10); - Map mergedRows = new HashMap<>(); - for (int i = 0; i < mergedRowCounts.length; i++) - { - long count = mergedRowCounts[i]; - if (count == 0) - continue; - - int rows = i + 1; - mergeSummary.append(String.format("%d:%d, ", rows, count)); - mergedRows.put(rows, count); - } - SystemKeyspace.updateCompactionHistory(keyspaceName, columnFamilyName, System.currentTimeMillis(), startSize, endSize, mergedRows); - return mergeSummary.toString(); - } - protected Directories getDirectories() { return cfs.getDirectories(); @@ -346,7 +633,7 @@ public static boolean getIsTransient(Set sstables) } - /* + /** * Checks if we have enough disk space to execute the compaction. Drops the largest sstable out of the Task until * there's enough space (in theory) to handle the compaction. Does not take into account space that will be taken by * other compactions. @@ -360,7 +647,7 @@ protected void buildCompactionCandidatesForAvailableDiskSpace(final Set nonExpiredSSTables = Sets.difference(transaction.originals(), fullyExpiredSSTables); - CompactionStrategyManager strategy = cfs.getCompactionStrategyManager(); + CompactionStrategy strategy = cfs.getCompactionStrategy(); int sstablesRemoved = 0; while(!nonExpiredSSTables.isEmpty()) @@ -429,4 +716,105 @@ public static long getMaxDataAge(Collection sstables) } return max; } + + private void debugLogCompactionSummaryInfo(UUID taskId, + long durationInNano, + long totalKeysWritten, + Collection newSStables, + CompactionProgress progress) + { + // log a bunch of statistics about the result and save to system table compaction_history + long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano); + + long totalMergedPartitions = 0; + long[] mergedPartitionCounts = progress.partitionsHistogram(); + StringBuilder mergeSummary = new StringBuilder(mergedPartitionCounts.length * 10); + mergeSummary.append('{'); + for (int i = 0; i < mergedPartitionCounts.length; i++) + { + long mergedPartitionCount = mergedPartitionCounts[i]; + if (mergedPartitionCount != 0) + { + totalMergedPartitions += mergedPartitionCount * (i + 1); + mergeSummary.append(i).append(':').append(mergedPartitionCount).append(", "); + } + } + mergeSummary.append('}'); + + StringBuilder newSSTableNames = new StringBuilder(newSStables.size() * 100); + for (SSTableReader reader : newSStables) + newSSTableNames.append(reader.descriptor.baseFilename()).append(","); + logger.debug("Compacted ({}) {} sstables to [{}] to level={}. {} to {} (~{}% of original) in {}ms. " + + "Read Throughput = {}, Write Throughput = {}, Row Throughput = ~{}/s, Partition Throughput = ~{}/s." + + " {} total partitions merged to {}. Partition merge counts were {}.", + taskId, + transaction.originals().size(), + newSSTableNames.toString(), + getLevel(), + prettyPrintMemory(progress.adjustedInputDiskSize()), + prettyPrintMemory(progress.outputDiskSize()), + (int) (progress.sizeRatio() * 100), + dTime, + prettyPrintMemoryPerSecond(progress.adjustedInputDiskSize(), durationInNano), + prettyPrintMemoryPerSecond(progress.outputDiskSize(), durationInNano), + progress.rowsRead() / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1), + (int) progress.partitionsRead() / (TimeUnit.NANOSECONDS.toSeconds(progress.durationInNanos()) + 1), + totalMergedPartitions, + totalKeysWritten, + mergeSummary.toString()); + } + + private void debugLogCompactingMessage(UUID taskId) + { + Set originals = transaction.originals(); + StringBuilder ssTableLoggerMsg = new StringBuilder(originals.size() * 100); + ssTableLoggerMsg.append("Compacting (").append(taskId).append(')').append(" ["); + for (SSTableReader sstr : originals) + { + ssTableLoggerMsg.append(sstr.getFilename()) + .append(":level=") + .append(sstr.getSSTableLevel()) + .append(", "); + } + ssTableLoggerMsg.append("]"); + + logger.debug(ssTableLoggerMsg.toString()); + } + + + private static void updateCompactionHistory(UUID id, + String keyspaceName, + String columnFamilyName, + CompactionProgress progress) + { + long[] mergedPartitionsHistogram = progress.partitionsHistogram(); + Map mergedPartitions = new HashMap<>(mergedPartitionsHistogram.length); + for (int i = 0; i < mergedPartitionsHistogram.length; i++) + { + long count = mergedPartitionsHistogram[i]; + if (count == 0) + continue; + + int rows = i + 1; + mergedPartitions.put(rows, count); + } + SystemKeyspace.updateCompactionHistory(id, + keyspaceName, + columnFamilyName, + System.currentTimeMillis(), + progress.adjustedInputDiskSize(), + progress.outputDiskSize(), + mergedPartitions); + } + + private void traceLogCompactionSummaryInfo(long totalKeysWritten, + long estimatedKeys, + CompactionProgress progress) + { + logger.trace("CF Total Bytes Compacted: {}", prettyPrintMemory(addToTotalBytesCompacted(progress.outputDiskSize()))); + logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", + totalKeysWritten, + estimatedKeys, + ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten)); + } } diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java index ab2b6ae32852..dcfc657e0f37 100644 --- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java @@ -17,20 +17,27 @@ */ package org.apache.cassandra.db.compaction; -import java.util.*; -import java.util.concurrent.TimeUnit; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; -import com.google.common.collect.*; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.collect.PeekingIterator; +import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -43,24 +50,24 @@ * @deprecated in favour of {@link TimeWindowCompactionStrategy} */ @Deprecated -public class DateTieredCompactionStrategy extends AbstractCompactionStrategy +public class DateTieredCompactionStrategy extends LegacyAbstractCompactionStrategy.WithSSTableList { private static final Logger logger = LoggerFactory.getLogger(DateTieredCompactionStrategy.class); - private final DateTieredCompactionStrategyOptions options; + private final DateTieredCompactionStrategyOptions dtOptions; protected volatile int estimatedRemainingTasks; private final Set sstables = new HashSet<>(); private long lastExpiredCheck; private final SizeTieredCompactionStrategyOptions stcsOptions; - public DateTieredCompactionStrategy(ColumnFamilyStore cfs, Map options) + public DateTieredCompactionStrategy(CompactionStrategyFactory factory, Map options) { - super(cfs, options); + super(factory, options); this.estimatedRemainingTasks = 0; - this.options = new DateTieredCompactionStrategyOptions(options); - if (!options.containsKey(AbstractCompactionStrategy.TOMBSTONE_COMPACTION_INTERVAL_OPTION) && !options.containsKey(AbstractCompactionStrategy.TOMBSTONE_THRESHOLD_OPTION)) + this.dtOptions = new DateTieredCompactionStrategyOptions(options); + if (!options.containsKey(CompactionStrategyOptions.TOMBSTONE_COMPACTION_INTERVAL_OPTION) && !options.containsKey(CompactionStrategyOptions.TOMBSTONE_THRESHOLD_OPTION)) { - disableTombstoneCompactions = true; + super.options.setDisableTombstoneCompactions(true); logger.trace("Disabling tombstone compactions for DTCS"); } else @@ -69,60 +76,31 @@ public DateTieredCompactionStrategy(ColumnFamilyStore cfs, Map o this.stcsOptions = new SizeTieredCompactionStrategyOptions(options); } - @Override - @SuppressWarnings("resource") - public AbstractCompactionTask getNextBackgroundTask(int gcBefore) - { - List previousCandidate = null; - while (true) - { - List latestBucket = getNextBackgroundSSTables(gcBefore); - - if (latestBucket.isEmpty()) - return null; - - // Already tried acquiring references without success. It means there is a race with - // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager - if (latestBucket.equals(previousCandidate)) - { - logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + - "unless it happens frequently, in which case it must be reported. Will retry later.", - latestBucket); - return null; - } - - LifecycleTransaction modifier = cfs.getTracker().tryModify(latestBucket, OperationType.COMPACTION); - if (modifier != null) - return new CompactionTask(cfs, modifier, gcBefore); - previousCandidate = latestBucket; - } - } - /** * * @param gcBefore * @return */ - private synchronized List getNextBackgroundSSTables(final int gcBefore) + protected synchronized List getNextBackgroundSSTables(final int gcBefore) { Set uncompacting; synchronized (sstables) { if (sstables.isEmpty()) - return Collections.emptyList(); + return ImmutableList.of(); - uncompacting = ImmutableSet.copyOf(filter(cfs.getUncompactingSSTables(), sstables::contains)); + uncompacting = ImmutableSet.copyOf(cfs.getNoncompactingSSTables(sstables)); } Set expired = Collections.emptySet(); // we only check for expired sstables every 10 minutes (by default) due to it being an expensive operation - if (System.currentTimeMillis() - lastExpiredCheck > options.expiredSSTableCheckFrequency) + if (System.currentTimeMillis() - lastExpiredCheck > dtOptions.expiredSSTableCheckFrequency) { // Find fully expired SSTables. Those will be included no matter what. expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, cfs.getOverlappingLiveSSTables(uncompacting), gcBefore); lastExpiredCheck = System.currentTimeMillis(); } - Set candidates = Sets.newHashSet(filterSuspectSSTables(uncompacting)); + Set candidates = Sets.newHashSet(Iterables.filter(uncompacting, sstable -> !sstable.isMarkedSuspect())); List compactionCandidates = new ArrayList<>(getNextNonExpiredSSTables(Sets.difference(candidates, expired), gcBefore)); if (!expired.isEmpty()) @@ -159,17 +137,17 @@ private List getNextNonExpiredSSTables(Iterable no private List getCompactionCandidates(Iterable candidateSSTables, long now, int base) { - Iterable candidates = filterOldSSTables(Lists.newArrayList(candidateSSTables), options.maxSSTableAge, now); + Iterable candidates = filterOldSSTables(Lists.newArrayList(candidateSSTables), dtOptions.maxSSTableAge, now); - List> buckets = getBuckets(createSSTableAndMinTimestampPairs(candidates), options.baseTime, base, now, options.maxWindowSize); + List> buckets = getBuckets(createSSTableAndMinTimestampPairs(candidates), dtOptions.baseTime, base, now, dtOptions.maxWindowSize); logger.debug("Compaction buckets are {}", buckets); updateEstimatedCompactionsByTasks(buckets); List mostInteresting = newestBucket(buckets, cfs.getMinimumCompactionThreshold(), cfs.getMaximumCompactionThreshold(), now, - options.baseTime, - options.maxWindowSize, + dtOptions.baseTime, + dtOptions.maxWindowSize, stcsOptions); if (!mostInteresting.isEmpty()) return mostInteresting; @@ -223,22 +201,47 @@ public static List> createSSTableAndMinTimestampPairs( return sstableMinTimestampPairs; } + public void replaceSSTables(Collection removed, Collection added) + { + synchronized (sstables) + { + for (SSTableReader remove : removed) + removeSSTable(remove); + addSSTables(added); + } + } + @Override - public synchronized void addSSTable(SSTableReader sstable) + public void addSSTable(SSTableReader sstable) { - sstables.add(sstable); + synchronized (sstables) + { + sstables.add(sstable); + } + } + + @Override + public void removeSSTable(SSTableReader sstable) + { + synchronized (sstables) + { + sstables.remove(sstable); + } } @Override - public synchronized void removeSSTable(SSTableReader sstable) + void removeDeadSSTables() { - sstables.remove(sstable); + removeDeadSSTables(sstables); } @Override - protected Set getSSTables() + public Set getSSTables() { - return ImmutableSet.copyOf(sstables); + synchronized (sstables) + { + return ImmutableSet.copyOf(sstables); + } } /** @@ -365,12 +368,12 @@ private void updateEstimatedCompactionsByTasks(List> tasks) int n = 0; for (List bucket : tasks) { - for (List stcsBucket : getSTCSBuckets(bucket, stcsOptions)) + for (List stcsBucket : getSTCSBuckets(bucket, stcsOptions, cfs.getMinimumCompactionThreshold(), cfs.getMaximumCompactionThreshold())) if (stcsBucket.size() >= cfs.getMinimumCompactionThreshold()) n += Math.ceil((double)stcsBucket.size() / cfs.getMaximumCompactionThreshold()); } estimatedRemainingTasks = n; - cfs.getCompactionStrategyManager().compactionLogger.pending(this, n); + getCompactionLogger().pending(this, n); } @@ -392,7 +395,7 @@ static List newestBucket(List> buckets, int m boolean inFirstWindow = incomingWindow.onTarget(bucket.get(0).getMinTimestamp()); if (bucket.size() >= minThreshold || (bucket.size() >= 2 && !inFirstWindow)) { - List stcsSSTables = getSSTablesForSTCS(bucket, inFirstWindow ? minThreshold : 2, maxThreshold, stcsOptions); + List stcsSSTables = getSSTablesForSTCS(bucket, stcsOptions, inFirstWindow ? minThreshold : 2, maxThreshold); if (!stcsSSTables.isEmpty()) return stcsSSTables; } @@ -400,49 +403,26 @@ static List newestBucket(List> buckets, int m return Collections.emptyList(); } - private static List getSSTablesForSTCS(Collection sstables, int minThreshold, int maxThreshold, SizeTieredCompactionStrategyOptions stcsOptions) + private static List getSSTablesForSTCS(Collection sstables, SizeTieredCompactionStrategyOptions stcsOptions, int minThreshold, int maxThreshold) { - List s = SizeTieredCompactionStrategy.mostInterestingBucket(getSTCSBuckets(sstables, stcsOptions), minThreshold, maxThreshold); + SizeTieredCompactionStrategy.SizeTieredBuckets sizeTieredBuckets = new SizeTieredCompactionStrategy.SizeTieredBuckets(sstables, + stcsOptions, + minThreshold, + maxThreshold); + + sizeTieredBuckets.aggregate(); + List s = new ArrayList<>(CompactionAggregate.getSelected(sizeTieredBuckets.getAggregates()).sstables); logger.debug("Got sstables {} for STCS from {}", s, sstables); return s; } - private static List> getSTCSBuckets(Collection sstables, SizeTieredCompactionStrategyOptions stcsOptions) - { - List> pairs = SizeTieredCompactionStrategy.createSSTableAndLengthPairs(AbstractCompactionStrategy.filterSuspectSSTables(sstables)); - return SizeTieredCompactionStrategy.getBuckets(pairs, - stcsOptions.bucketHigh, - stcsOptions.bucketLow, - stcsOptions.minSSTableSize); - } - - @Override - @SuppressWarnings("resource") - public synchronized Collection getMaximalTask(int gcBefore, boolean splitOutput) - { - Iterable filteredSSTables = filterSuspectSSTables(sstables); - if (Iterables.isEmpty(filteredSSTables)) - return null; - LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION); - if (txn == null) - return null; - return Collections.singleton(new CompactionTask(cfs, txn, gcBefore)); - } - - @Override - @SuppressWarnings("resource") - public synchronized AbstractCompactionTask getUserDefinedTask(Collection sstables, int gcBefore) + private static List> getSTCSBuckets(Collection sstables, SizeTieredCompactionStrategyOptions stcsOptions, int minThreshold, int maxThreshold) { - assert !sstables.isEmpty(); // checked for by CM.submitUserDefined - - LifecycleTransaction modifier = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); - if (modifier == null) - { - logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables); - return null; - } - - return new CompactionTask(cfs, modifier, gcBefore).setUserDefined(true); + SizeTieredCompactionStrategy.SizeTieredBuckets sizeTieredBuckets = new SizeTieredCompactionStrategy.SizeTieredBuckets(sstables, + stcsOptions, + minThreshold, + maxThreshold); + return sizeTieredBuckets.buckets(); } public int getEstimatedRemainingTasks() @@ -471,7 +451,7 @@ public Collection> groupSSTablesForAntiCompaction(Coll public static Map validateOptions(Map options) throws ConfigurationException { - Map uncheckedOptions = AbstractCompactionStrategy.validateOptions(options); + Map uncheckedOptions = CompactionStrategyOptions.validateOptions(options); uncheckedOptions = DateTieredCompactionStrategyOptions.validateOptions(options, uncheckedOptions); uncheckedOptions.remove(CompactionParams.Option.MIN_THRESHOLD.toString()); @@ -482,33 +462,6 @@ public static Map validateOptions(Map options) t return uncheckedOptions; } - public CompactionLogger.Strategy strategyLogger() - { - return new CompactionLogger.Strategy() - { - public JsonNode sstable(SSTableReader sstable) - { - ObjectNode node = JsonNodeFactory.instance.objectNode(); - node.put("min_timestamp", sstable.getMinTimestamp()); - node.put("max_timestamp", sstable.getMaxTimestamp()); - return node; - } - - public JsonNode options() - { - ObjectNode node = JsonNodeFactory.instance.objectNode(); - TimeUnit resolution = DateTieredCompactionStrategy.this.options.timestampResolution; - node.put(DateTieredCompactionStrategyOptions.TIMESTAMP_RESOLUTION_KEY, - resolution.toString()); - node.put(DateTieredCompactionStrategyOptions.BASE_TIME_KEY, - resolution.toSeconds(DateTieredCompactionStrategy.this.options.baseTime)); - node.put(DateTieredCompactionStrategyOptions.MAX_WINDOW_SIZE_KEY, - resolution.toSeconds(DateTieredCompactionStrategy.this.options.maxWindowSize)); - return node; - } - }; - } - public String toString() { return String.format("DateTieredCompactionStrategy[%s/%s]", diff --git a/src/java/org/apache/cassandra/db/compaction/LegacyAbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LegacyAbstractCompactionStrategy.java new file mode 100644 index 000000000000..2c307a5e71d5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/LegacyAbstractCompactionStrategy.java @@ -0,0 +1,344 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; + +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; + +/** + * Pluggable compaction strategy determines how SSTables get merged. + * + * There are two main goals: + * - perform background compaction constantly as needed; this typically makes a tradeoff between + * i/o done by compaction, and merging done at read time. + * - perform a full (maximum possible) compaction if requested by the user + */ +abstract class LegacyAbstractCompactionStrategy extends AbstractCompactionStrategy +{ + protected LegacyAbstractCompactionStrategy(CompactionStrategyFactory factory, Map options) + { + super(factory, new BackgroundCompactions(factory.getCfs()), options); + assert factory != null; + } + + /** + * Helper base class for strategies that provide CompactionAggregates, implementing the typical + * getNextBackgroundTasks logic based on a getNextBackgroundAggregate method. + */ + protected static abstract class WithAggregates extends LegacyAbstractCompactionStrategy + { + protected WithAggregates(CompactionStrategyFactory factory, Map options) + { + super(factory, options); + } + + @Override + @SuppressWarnings("resource") + public Collection getNextBackgroundTasks(int gcBefore) + { + CompactionPick previous = null; + while (true) + { + CompactionAggregate compaction = getNextBackgroundAggregate(gcBefore); + if (compaction == null || compaction.isEmpty()) + return ImmutableList.of(); + + // Already tried acquiring references without success. It means there is a race with + // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager + if (compaction.getSelected().equals(previous)) + { + logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + + "unless it happens frequently, in which case it must be reported. Will retry later.", + compaction.getSelected()); + return ImmutableList.of(); + } + + LifecycleTransaction transaction = dataTracker.tryModify(compaction.getSelected().sstables, OperationType.COMPACTION); + if (transaction != null) + { + backgroundCompactions.setSubmitted(this, transaction.opId(), compaction); + return ImmutableList.of(createCompactionTask(gcBefore, transaction, compaction)); + } + + // Getting references to the sstables failed. This may be because we tried to compact sstables that are + // no longer present (due to races in getting the notification), or because we still haven't + // received any replace notifications. Remove any non-live sstables we track and try again. + removeDeadSSTables(); + + previous = compaction.getSelected(); + } + } + + /** + * Select the next compaction to perform. This method is typically synchronized. + */ + protected abstract CompactionAggregate getNextBackgroundAggregate(int gcBefore); + + protected AbstractCompactionTask createCompactionTask(final int gcBefore, LifecycleTransaction txn, CompactionAggregate compaction) + { + return new CompactionTask(cfs, txn, gcBefore, false, this); + } + + /** + * Get the estimated remaining compactions. Strategies that implement {@link WithAggregates} can delegate this + * to {@link BackgroundCompactions} because they set the pending aggregates as background compactions but legacy + * strategies that do not support aggregates must implement this method. + *

    + * @return the number of background tasks estimated to still be needed for this strategy + */ + @Override + public int getEstimatedRemainingTasks() + { + return backgroundCompactions.getEstimatedRemainingTasks(); + } + } + + /** + * Helper base class for (older, deprecated) strategies that provide a list of tables to compact, implementing the + * typical getNextBackgroundTask logic based on a getNextBackgroundSSTables method. + */ + protected static abstract class WithSSTableList extends LegacyAbstractCompactionStrategy + { + protected WithSSTableList(CompactionStrategyFactory factory, Map options) + { + super(factory, options); + } + + @Override + @SuppressWarnings("resource") + public Collection getNextBackgroundTasks(int gcBefore) + { + List previousCandidate = null; + while (true) + { + List latestBucket = getNextBackgroundSSTables(gcBefore); + + if (latestBucket.isEmpty()) + return ImmutableList.of(); + + // Already tried acquiring references without success. It means there is a race with + // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager + if (latestBucket.equals(previousCandidate)) + { + logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + + "unless it happens frequently, in which case it must be reported. Will retry later.", + latestBucket); + return ImmutableList.of(); + } + + LifecycleTransaction modifier = dataTracker.tryModify(latestBucket, OperationType.COMPACTION); + if (modifier != null) + return ImmutableList.of(createCompactionTask(gcBefore, modifier, false, false)); + + // Getting references to the sstables failed. This may be because we tried to compact sstables that are + // no longer present (due to races in getting the notification), or because we still haven't + // received any replace notifications. Remove any non-live sstables we track and try again. + removeDeadSSTables(); + + previousCandidate = latestBucket; + } + } + + /** + * Select the next tables to compact. This method is typically synchronized. + */ + protected abstract List getNextBackgroundSSTables(final int gcBefore); + } + + /** + * Replaces sstables in the compaction strategy + * + * Note that implementations must be able to handle duplicate notifications here (that removed are already gone and + * added have already been added) + * */ + public abstract void replaceSSTables(Collection removed, Collection added); + + /** + * Adds sstable, note that implementations must handle duplicate notifications here (added already being in the compaction strategy) + */ + abstract void addSSTable(SSTableReader added); + + /** + * Adds sstables, note that implementations must handle duplicate notifications here (added already being in the compaction strategy) + */ + public synchronized void addSSTables(Iterable added) + { + for (SSTableReader sstable : added) + addSSTable(sstable); + } + + /** + * Removes sstable from the strategy, implementations must be able to handle the sstable having already been removed. + */ + abstract void removeSSTable(SSTableReader sstable); + + /** + * Removes sstables from the strategy, implementations must be able to handle the sstables having already been removed. + */ + public void removeSSTables(Iterable removed) + { + for (SSTableReader sstable : removed) + removeSSTable(sstable); + } + + /** + * Remove any tracked sstable that is no longer in the live set. Note that because we get notifications after the + * tracker is modified, anything we know of must be already in the live set. If it is not, it has been removed + * from there, and we either haven't received the removal notification yet, or we did and we messed it up (i.e. + * we got it before the addition). The former is transient, but the latter can cause persistent problems, including + * fully stopping compaction. In any case, we should remove any such sstables. + * There is a special-case implementation of this in LeveledManifest. + */ + abstract void removeDeadSSTables(); + + void removeDeadSSTables(Iterable sstables) + { + synchronized (sstables) + { + int removed = 0; + Set liveSet = cfs.getLiveSSTables(); + for (Iterator it = sstables.iterator(); it.hasNext(); ) + { + SSTableReader sstable = it.next(); + if (!liveSet.contains(sstable)) + { + it.remove(); + ++removed; + } + } + + if (removed > 0) + logger.debug("Removed {} dead sstables from the compactions tracked list.", removed); + } + } + + public synchronized CompactionTasks getMaximalTasks(int gcBefore, boolean splitOutput) + { + removeDeadSSTables(); + return super.getMaximalTasks(gcBefore, splitOutput); + } + + /** + * Called when the metadata has changed for an sstable - for example if the level changed + * + * Not called when repair status changes (which is also metadata), because this results in the + * sstable getting removed from the compaction strategy instance. + * + * This is only needed by the LCS manifest from what I could see. + */ + void metadataChanged(StatsMetadata oldMetadata, SSTableReader sstable) + { + } + + /** + * Select a table for tombstone-removing compaction from the given set. Returns null if no table is suitable. + */ + @Nullable + CompactionAggregate makeTombstoneCompaction(int gcBefore, + Iterable candidates, + Function, SSTableReader> selector) + { + List sstablesWithTombstones = new ArrayList<>(); + for (SSTableReader sstable : candidates) + { + if (worthDroppingTombstones(sstable, gcBefore)) + sstablesWithTombstones.add(sstable); + } + if (sstablesWithTombstones.isEmpty()) + return null; + + final SSTableReader sstable = selector.apply(sstablesWithTombstones); + return CompactionAggregate.createForTombstones(sstable); + } + + /** + * Check if given sstable is worth dropping tombstones at gcBefore. + * Check is skipped if tombstone_compaction_interval time does not elapse since sstable creation and returns false. + * + * @param sstable SSTable to check + * @param gcBefore time to drop tombstones + * @return true if given sstable's tombstones are expected to be removed + */ + protected boolean worthDroppingTombstones(SSTableReader sstable, int gcBefore) + { + if (options.isDisableTombstoneCompactions() || CompactionController.NEVER_PURGE_TOMBSTONES || cfs.getNeverPurgeTombstones()) + return false; + // since we use estimations to calculate, there is a chance that compaction will not drop tombstones actually. + // if that happens we will end up in infinite compaction loop, so first we check enough if enough time has + // elapsed since SSTable created. + if (System.currentTimeMillis() < sstable.getCreationTimeFor(Component.DATA) + options.getTombstoneCompactionInterval() * 1000) + return false; + + double droppableRatio = sstable.getEstimatedDroppableTombstoneRatio(gcBefore); + if (droppableRatio <= options.getTombstoneThreshold()) + return false; + + //sstable range overlap check is disabled. See CASSANDRA-6563. + if (options.isUncheckedTombstoneCompaction()) + return true; + + Collection overlaps = cfs.getOverlappingLiveSSTables(Collections.singleton(sstable)); + if (overlaps.isEmpty()) + { + // there is no overlap, tombstones are safely droppable + return true; + } + else if (CompactionController.getFullyExpiredSSTables(cfs, Collections.singleton(sstable), overlaps, gcBefore).size() > 0) + { + return true; + } + else + { + // what percentage of columns do we expect to compact outside of overlap? + if (sstable.getIndexSummarySize() < 2) + { + // we have too few samples to estimate correct percentage + return false; + } + // first, calculate estimated keys that do not overlap + long keys = sstable.estimatedKeys(); + Set> ranges = new HashSet>(overlaps.size()); + for (SSTableReader overlap : overlaps) + ranges.add(new Range<>(overlap.first.getToken(), overlap.last.getToken())); + long remainingKeys = keys - sstable.estimatedKeysForRanges(ranges); + // next, calculate what percentage of columns we have within those keys + long columns = sstable.getEstimatedCellPerPartitionCount().mean() * remainingKeys; + double remainingColumnsRatio = ((double) columns) / (sstable.getEstimatedCellPerPartitionCount().count() * sstable.getEstimatedCellPerPartitionCount().mean()); + + // return if we still expect to have droppable tombstones in rest of columns + return remainingColumnsRatio * droppableRatio > options.getTombstoneThreshold(); + } + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStatistics.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStatistics.java new file mode 100644 index 000000000000..15fd1b725470 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStatistics.java @@ -0,0 +1,169 @@ +/* + * 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.compaction; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; + +/** + * The statistics for leveled compaction. + *

    + * Implements serializable to allow structured info to be returned via JMX. + */ +public class LeveledCompactionStatistics extends CompactionAggregateStatistics +{ + private static final Collection HEADER = ImmutableList.copyOf(Iterables.concat(ImmutableList.of("Level", "Score"), + CompactionAggregateStatistics.HEADER, + ImmutableList.of("Read: Tot/Prev/Next", + "Written: Tot/New", + "WA (tot_written/read_prev)"))); + + private static final long serialVersionUID = 3695927592357744816L; + + /** The current level */ + private final int level; + + /** The score of this level */ + private final double score; + + /** + * How many more compactions this level is expected to perform. This is required because for LCS we cannot + * easily identify candidate sstables to put into the pending picks. + */ + private final int pendingCompactions; + + /** + * Bytes read from the current level (N) during compaction between levels N and N+1. Note that {@link #readBytes} + * includes bytes read from both the current level (N) and the target level (N+1). + */ + private final long readLevel; + + /** + * Additional RocksDB metrics we may want to consider: + * Moved(GB): Bytes moved to level N+1 during compaction. In this case there is no IO other than updating the manifest to indicate that a file which used to be in level X is now in level Y + * Rd(MB/s): The rate at which data is read during compaction between levels N and N+1. This is (Read(GB) * 1024) / duration where duration is the time for which compactions are in progress from level N to N+1. + * Wr(MB/s): The rate at which data is written during compaction. See Rd(MB/s). + * Rn(cnt): Total files read from level N during compaction between levels N and N+1 + * Rnp1(cnt): Total files read from level N+1 during compaction between levels N and N+1 + * Wnp1(cnt): Total files written to level N+1 during compaction between levels N and N+1 + * Wnew(cnt): (Wnp1(cnt) - Rnp1(cnt)) -- Increase in file count as result of compaction between levels N and N+1 + * Comp(sec): Total time spent doing compactions between levels N and N+1 + * Comp(cnt): Total number of compactions between levels N and N+1 + * Avg(sec): Average time per compaction between levels N and N+1 + * Stall(sec): Total time writes were stalled because level N+1 was uncompacted (compaction score was high) + * Stall(cnt): Total number of writes stalled because level N+1 was uncompacted + * Avg(ms): Average time in milliseconds a write was stalled because level N+1 was uncompacted + * KeyIn: number of records compared during compaction + * KeyDrop: number of records dropped (not written out) during compaction + */ + + public LeveledCompactionStatistics(CompactionAggregateStatistics base, + int level, + double score, + int pendingCompactions, + long readLevel) + { + super(base); + this.level = level; + this.score = score; + this.pendingCompactions = pendingCompactions; + this.readLevel = readLevel; + } + + /** The number of compactions that are either pending or in progress */ + @Override + @JsonProperty + public int numCompactions() + { + return numCompactions + pendingCompactions; + } + + /** The current level */ + @JsonProperty + public int level() + { + return level; + } + + /** The score of a level is the level size in bytes of all its files dived by the ideal + * level size if applicable, or zero for tiered strategies */ + @JsonProperty + public double score() + { + return score; + } + + /** + * Bytes read from the current level (N) during compaction between levels N and N+1. Note that + * {@link #read()} includes bytes read from both the current level (N) and the target level (N+1). + */ @JsonProperty + public long readLevel() + { + return readLevel; + } + + /** Uncompressed bytes read from the next level (N+1) during compaction between levels N and N+1 */ + @JsonProperty + public long readNext() + { + return readBytes - readLevel; + } + + /** Uncompressed bytes written to level N+1, calculated as total bytes written - bytes read from N+1 */ + @JsonProperty + public long writtenNew() + { + return writtenBytes - readNext(); + } + + /** W-Amp: total bytes written divided by the bytes read from level N. */ + @JsonProperty + public double writeAmpl() + { + return readLevel() > 0 ? (double) writtenBytes / readLevel() : Double.NaN; + } + + @Override + protected Collection header() + { + return HEADER; + } + + @Override + protected Collection data() + { + List data = new ArrayList<>(HEADER.size()); + data.add(Integer.toString(level())); + data.add(String.format("%.3f", score())); + + data.addAll(super.data()); + + data.add(toString(read()) + '/' + toString(readLevel()) + '/' + toString(readNext())); + data.add(toString(written()) + '/' + toString(writtenNew())); + data.add(String.format("%.3f", writeAmpl())); + + return data; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java index dd7c9dfcff79..1c6339353f25 100644 --- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java @@ -17,38 +17,48 @@ */ package org.apache.cassandra.db.compaction; -import java.util.*; - +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.*; +import com.google.common.collect.AbstractIterator; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; import com.google.common.primitives.Doubles; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.cassandra.io.sstable.metadata.StatsMetadata; -import org.apache.cassandra.schema.CompactionParams; -import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.config.Config; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.ScannerList; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.schema.TableMetadata; -public class LeveledCompactionStrategy extends AbstractCompactionStrategy +public class LeveledCompactionStrategy extends LegacyAbstractCompactionStrategy.WithAggregates { private static final Logger logger = LoggerFactory.getLogger(LeveledCompactionStrategy.class); - private static final String SSTABLE_SIZE_OPTION = "sstable_size_in_mb"; + static final String SSTABLE_SIZE_OPTION = "sstable_size_in_mb"; private static final boolean tolerateSstableSize = Boolean.getBoolean(Config.PROPERTY_PREFIX + "tolerate_sstable_size"); - private static final String LEVEL_FANOUT_SIZE_OPTION = "fanout_size"; + static final String LEVEL_FANOUT_SIZE_OPTION = "fanout_size"; private static final String SINGLE_SSTABLE_UPLEVEL_OPTION = "single_sstable_uplevel"; public static final int DEFAULT_LEVEL_FANOUT_SIZE = 10; @@ -58,9 +68,9 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy private final int levelFanoutSize; private final boolean singleSSTableUplevel; - public LeveledCompactionStrategy(ColumnFamilyStore cfs, Map options) + public LeveledCompactionStrategy(CompactionStrategyFactory factory, Map options) { - super(cfs, options); + super(factory, options); int configuredMaxSSTableSize = 160; int configuredLevelFanoutSize = DEFAULT_LEVEL_FANOUT_SIZE; boolean configuredSingleSSTableUplevel = false; @@ -74,10 +84,10 @@ public LeveledCompactionStrategy(ColumnFamilyStore cfs, Map opti { if (configuredMaxSSTableSize >= 1000) logger.warn("Max sstable size of {}MB is configured for {}.{}; having a unit of compaction this large is probably a bad idea", - configuredMaxSSTableSize, cfs.name, cfs.getTableName()); + configuredMaxSSTableSize, cfs.getKeyspaceName(), cfs.getTableName()); if (configuredMaxSSTableSize < 50) logger.warn("Max sstable size of {}MB is configured for {}.{}. Testing done for CASSANDRA-5727 indicates that performance improves up to 160MB", - configuredMaxSSTableSize, cfs.name, cfs.getTableName()); + configuredMaxSSTableSize, cfs.getKeyspaceName(), cfs.getTableName()); } } @@ -99,7 +109,7 @@ public LeveledCompactionStrategy(ColumnFamilyStore cfs, Map opti logger.trace("Created {}", manifest); } - public int getLevelSize(int i) + int getLevelSize(int i) { return manifest.getLevelSize(i); } @@ -109,6 +119,12 @@ public int[] getAllLevelSize() return manifest.getAllLevelSize(); } + @Override + public int[] getSSTableCountPerLevel() + { + return manifest.getSSTableCountPerLevel(); + } + @Override public void startup() { @@ -116,98 +132,62 @@ public void startup() super.startup(); } - /** - * the only difference between background and maximal in LCS is that maximal is still allowed - * (by explicit user request) even when compaction is disabled. - */ - @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute - public AbstractCompactionTask getNextBackgroundTask(int gcBefore) + @Override + protected CompactionAggregate getNextBackgroundAggregate(int gcBefore) { - Collection previousCandidate = null; - while (true) - { - OperationType op; - LeveledManifest.CompactionCandidate candidate = manifest.getCompactionCandidates(); - if (candidate == null) - { - // if there is no sstable to compact in standard way, try compacting based on droppable tombstone ratio - SSTableReader sstable = findDroppableSSTable(gcBefore); - if (sstable == null) - { - logger.trace("No compaction necessary for {}", this); - return null; - } - candidate = new LeveledManifest.CompactionCandidate(Collections.singleton(sstable), - sstable.getSSTableLevel(), - getMaxSSTableBytes()); - op = OperationType.TOMBSTONE_COMPACTION; - } - else - { - op = OperationType.COMPACTION; - } - - // Already tried acquiring references without success. It means there is a race with - // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager - if (candidate.sstables.equals(previousCandidate)) - { - logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + - "unless it happens frequently, in which case it must be reported. Will retry later.", - candidate.sstables); - return null; - } + CompactionAggregate.Leveled candidate = manifest.getCompactionCandidate(); + backgroundCompactions.setPending(this, manifest.getEstimatedTasks(candidate)); - LifecycleTransaction txn = cfs.getTracker().tryModify(candidate.sstables, OperationType.COMPACTION); - if (txn != null) - { - AbstractCompactionTask newTask; - if (!singleSSTableUplevel || op == OperationType.TOMBSTONE_COMPACTION || txn.originals().size() > 1) - newTask = new LeveledCompactionTask(cfs, txn, candidate.level, gcBefore, candidate.maxSSTableBytes, false); - else - newTask = new SingleSSTableLCSTask(cfs, txn, candidate.level); + if (candidate != null) + return candidate; - newTask.setCompactionType(op); - return newTask; - } - previousCandidate = candidate.sstables; - } + return findDroppableSSTable(gcBefore); } - @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute - public synchronized Collection getMaximalTask(int gcBefore, boolean splitOutput) + @Override + protected AbstractCompactionTask createCompactionTask(final int gcBefore, LifecycleTransaction txn, CompactionAggregate compaction) { - Iterable sstables = manifest.getSSTables(); + long maxxSSTableBytes; + int nextLevel; + OperationType op; + + if (compaction instanceof CompactionAggregate.TombstoneAggregate) + { + op = OperationType.TOMBSTONE_COMPACTION; + nextLevel = Iterables.getOnlyElement(compaction.selected.sstables).getSSTableLevel(); + maxxSSTableBytes = getMaxSSTableBytes(); // TODO: verify this is expected as it can split L0 tables + } + else + { + CompactionAggregate.Leveled candidate = (CompactionAggregate.Leveled) compaction; + op = OperationType.COMPACTION; + nextLevel = candidate.nextLevel; + maxxSSTableBytes = candidate.maxSSTableBytes; + } - Iterable filteredSSTables = filterSuspectSSTables(sstables); - if (Iterables.isEmpty(sstables)) - return null; - LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION); - if (txn == null) - return null; - return Arrays.asList(new LeveledCompactionTask(cfs, txn, 0, gcBefore, getMaxSSTableBytes(), true)); + AbstractCompactionTask newTask; + if (!singleSSTableUplevel || op == OperationType.TOMBSTONE_COMPACTION || txn.originals().size() > 1) + newTask = new LeveledCompactionTask(this, txn, nextLevel, gcBefore, maxxSSTableBytes, false); + else + newTask = new SingleSSTableLCSTask(this, txn, nextLevel); + + newTask.setCompactionType(op); + return newTask; } + @Override - @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute - public AbstractCompactionTask getUserDefinedTask(Collection sstables, int gcBefore) + protected AbstractCompactionTask createCompactionTask(final int gcBefore, LifecycleTransaction txn, boolean isMaximal, boolean splitOutput) { - - if (sstables.isEmpty()) - return null; - - LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); - if (transaction == null) - { - logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables); - return null; - } + Collection sstables = txn.originals(); int level = sstables.size() > 1 ? 0 : sstables.iterator().next().getSSTableLevel(); - return new LeveledCompactionTask(cfs, transaction, level, gcBefore, level == 0 ? Long.MAX_VALUE : getMaxSSTableBytes(), false); + long maxSSTableBytes = (level == 0 && !isMaximal) ? Long.MAX_VALUE : getMaxSSTableBytes(); + return new LeveledCompactionTask(this, txn, level, gcBefore, maxSSTableBytes, isMaximal); } @Override - public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes) + public AbstractCompactionTask createCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes) { assert txn.originals().size() > 0; int level = -1; @@ -219,7 +199,7 @@ public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gc if (level != sstable.getSSTableLevel()) level = 0; } - return new LeveledCompactionTask(cfs, txn, level, gcBefore, maxSSTableBytes, false); + return new LeveledCompactionTask(this, txn, level, gcBefore, maxSSTableBytes, false); } /** @@ -268,13 +248,6 @@ public Collection> groupSSTablesForAntiCompaction(Coll } - public int getEstimatedRemainingTasks() - { - int n = manifest.getEstimatedTasks(); - cfs.getCompactionStrategyManager().compactionLogger.pending(this, n); - return n; - } - public long getMaxSSTableBytes() { return maxSSTableSizeInMB * 1024L * 1024L; @@ -326,7 +299,7 @@ public ScannerList getScanners(Collection sstables, Collection sstables) manifest.addSSTables(sstables); } + @Override + void removeDeadSSTables() + { + manifest.removeDeadSSTables(); + } + @Override public void addSSTable(SSTableReader added) { @@ -372,7 +351,7 @@ public void removeSSTable(SSTableReader sstable) } @Override - protected Set getSSTables() + public Set getSSTables() { return manifest.getSSTables(); } @@ -384,6 +363,7 @@ private static class LeveledScanner extends AbstractIterator> ranges; private final List sstables; + private final int level; private final Iterator sstableIterator; private final long totalLength; private final long compressedLength; @@ -392,13 +372,14 @@ private static class LeveledScanner extends AbstractIterator sstables, Collection> ranges) + public LeveledScanner(TableMetadata metadata, Collection sstables, Collection> ranges, int level) { this.metadata = metadata; this.ranges = ranges; // add only sstables that intersect our range, and estimate how much data that involves this.sstables = new ArrayList<>(sstables.size()); + this.level = level; long length = 0; long cLength = 0; for (SSTableReader sstable : sstables) @@ -416,7 +397,7 @@ public LeveledScanner(TableMetadata metadata, Collection sstables totalLength = length; compressedLength = cLength; - Collections.sort(this.sstables, SSTableReader.sstableComparator); + Collections.sort(this.sstables, SSTableReader.firstKeyComparator); sstableIterator = this.sstables.iterator(); assert sstableIterator.hasNext(); // caller should check intersecting first SSTableReader currentSSTable = sstableIterator.next(); @@ -502,63 +483,43 @@ public Set getBackingSSTables() { return ImmutableSet.copyOf(sstables); } + + public int level() + { + return level; + } } @Override public String toString() { - return String.format("LCS@%d(%s)", hashCode(), cfs.name); + return String.format("LCS@%d(%s)", hashCode(), cfs.getTableName()); } - private SSTableReader findDroppableSSTable(final int gcBefore) + private CompactionAggregate findDroppableSSTable(final int gcBefore) { - level: + Comparator comparator = (o1, o2) -> { + double r1 = o1.getEstimatedDroppableTombstoneRatio(gcBefore); + double r2 = o2.getEstimatedDroppableTombstoneRatio(gcBefore); + return -1 * Doubles.compare(r1, r2); + }; + Function, SSTableReader> selector = list -> Collections.max(list, comparator); + Set compacting = dataTracker.getCompacting(); + for (int i = manifest.getLevelCount(); i >= 0; i--) { - if (manifest.getLevelSize(i) == 0) - continue; - // sort sstables by droppable ratio in descending order - List tombstoneSortedSSTables = manifest.getLevelSorted(i, (o1, o2) -> { - double r1 = o1.getEstimatedDroppableTombstoneRatio(gcBefore); - double r2 = o2.getEstimatedDroppableTombstoneRatio(gcBefore); - return -1 * Doubles.compare(r1, r2); - }); - - Set compacting = cfs.getTracker().getCompacting(); - for (SSTableReader sstable : tombstoneSortedSSTables) - { - if (sstable.getEstimatedDroppableTombstoneRatio(gcBefore) <= tombstoneThreshold) - continue level; - else if (!compacting.contains(sstable) && !sstable.isMarkedSuspect() && worthDroppingTombstones(sstable, gcBefore)) - return sstable; - } + CompactionAggregate tombstoneAggregate = makeTombstoneCompaction(gcBefore, + nonSuspectAndNotIn(manifest.getLevel(i), compacting), + selector); + if (tombstoneAggregate != null) + return tombstoneAggregate; } return null; } - public CompactionLogger.Strategy strategyLogger() - { - return new CompactionLogger.Strategy() - { - public JsonNode sstable(SSTableReader sstable) - { - ObjectNode node = JsonNodeFactory.instance.objectNode(); - node.put("level", sstable.getSSTableLevel()); - node.put("min_token", sstable.first.getToken().toString()); - node.put("max_token", sstable.last.getToken().toString()); - return node; - } - - public JsonNode options() - { - return null; - } - }; - } - public static Map validateOptions(Map options) throws ConfigurationException { - Map uncheckedOptions = AbstractCompactionStrategy.validateOptions(options); + Map uncheckedOptions = CompactionStrategyOptions.validateOptions(options); String size = options.containsKey(SSTABLE_SIZE_OPTION) ? options.get(SSTABLE_SIZE_OPTION) : "1"; try diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java index c6339374b453..cdf72f5847ba 100644 --- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java @@ -33,9 +33,9 @@ public class LeveledCompactionTask extends CompactionTask private final long maxSSTableBytes; private final boolean majorCompaction; - public LeveledCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int level, int gcBefore, long maxSSTableBytes, boolean majorCompaction) + public LeveledCompactionTask(LeveledCompactionStrategy strategy, LifecycleTransaction txn, int level, int gcBefore, long maxSSTableBytes, boolean majorCompaction) { - super(cfs, txn, gcBefore); + super(strategy.cfs, txn, gcBefore, false, strategy); this.level = level; this.maxSSTableBytes = maxSSTableBytes; this.majorCompaction = majorCompaction; diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledGenerations.java b/src/java/org/apache/cassandra/db/compaction/LeveledGenerations.java index 64027f2aa777..b33b244a35a6 100644 --- a/src/java/org/apache/cassandra/db/compaction/LeveledGenerations.java +++ b/src/java/org/apache/cassandra/db/compaction/LeveledGenerations.java @@ -73,9 +73,9 @@ class LeveledGenerations private final TreeSet [] levels = new TreeSet[MAX_LEVEL_COUNT - 1]; private static final Comparator nonL0Comparator = (o1, o2) -> { - int cmp = SSTableReader.sstableComparator.compare(o1, o2); + int cmp = SSTableReader.firstKeyComparator.compare(o1, o2); if (cmp == 0) - cmp = Ints.compare(o1.descriptor.generation, o2.descriptor.generation); + cmp = Ints.compare(o1.getGeneration(), o2.getGeneration()); return cmp; }; @@ -152,8 +152,8 @@ void addAll(Iterable readers) SSTableReader after = level.ceiling(sstable); SSTableReader before = level.floor(sstable); - if (before != null && before.last.compareTo(sstable.first) >= 0 || - after != null && after.first.compareTo(sstable.last) <= 0) + if (before != null && before.getLast().compareTo(sstable.getFirst()) >= 0 || + after != null && after.getFirst().compareTo(sstable.getLast()) <= 0) { sendToL0(sstable); } @@ -254,7 +254,7 @@ Iterator wrappingIterator(int lvl, SSTableReader lastCompactedSST while (tail.hasNext()) { SSTableReader potentialPivot = tail.peek(); - if (potentialPivot.first.compareTo(lastCompactedSSTable.last) > 0) + if (potentialPivot.getFirst().compareTo(lastCompactedSSTable.getLast()) > 0) { pivot = potentialPivot; break; @@ -312,7 +312,7 @@ private void maybeVerifyLevels() for (SSTableReader sstable : get(i)) { // no overlap: - assert prev == null || prev.last.compareTo(sstable.first) < 0; + assert prev == null || prev.getLast().compareTo(sstable.getFirst()) < 0; prev = sstable; // make sure it does not exist in any other level: for (int j = 0; j < levelCount(); j++) diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java index 7c865c7be1e7..cf4ca4c020fa 100644 --- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java +++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java @@ -25,11 +25,12 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import com.google.common.primitives.Ints; import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.lifecycle.Tracker; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableReader; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +40,6 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.Pair; import static org.apache.cassandra.db.compaction.LeveledGenerations.MAX_LEVEL_COUNT; @@ -51,7 +51,13 @@ public class LeveledManifest * if we have more than MAX_COMPACTING_L0 sstables in L0, we will run a round of STCS with at most * cfs.getMaxCompactionThreshold() sstables. */ - private static final int MAX_COMPACTING_L0 = 32; + @VisibleForTesting + static final int MAX_COMPACTING_L0 = 32; + + /** + * The maximum number of sstables in L0 for calculating the maximum number of bytes in L0. + */ + static final int MAX_SSTABLES_L0 = 4; /** * If we go this many rounds without compacting @@ -61,6 +67,7 @@ public class LeveledManifest private static final int NO_COMPACTION_LIMIT = 25; private final ColumnFamilyStore cfs; + private final Tracker dataTracker; private final LeveledGenerations generations; @@ -73,6 +80,7 @@ public class LeveledManifest LeveledManifest(ColumnFamilyStore cfs, int maxSSTableSizeInMB, int fanoutSize, SizeTieredCompactionStrategyOptions options) { this.cfs = cfs; + this.dataTracker = cfs.getTracker(); this.maxSSTableSizeInBytes = maxSSTableSizeInMB * 1024L * 1024L; this.options = options; this.levelFanoutSize = fanoutSize; @@ -151,7 +159,33 @@ public synchronized void replace(Collection removed, Collection liveSet = cfs.getLiveSSTables(); + + for (int i = 0; i < generations.levelCount(); i++) + { + Iterator it = generations.get(i).iterator(); + while (it.hasNext()) + { + SSTableReader sstable = it.next(); + if (!liveSet.contains(sstable)) + { + it.remove(); + ++removed; + } + } + } + + if (removed > 0) + logger.debug("Removed {} dead sstables from the compactions tracked list.", removed); } private String toString(Collection sstables) @@ -159,9 +193,9 @@ private String toString(Collection sstables) StringBuilder builder = new StringBuilder(); for (SSTableReader sstable : sstables) { - builder.append(sstable.descriptor.cfname) + builder.append(sstable.getColumnFamilyName()) .append('-') - .append(sstable.descriptor.generation) + .append(sstable.getGeneration()) .append("(L") .append(sstable.getSSTableLevel()) .append("), "); @@ -177,7 +211,7 @@ public long maxBytesForLevel(int level, long maxSSTableSizeInBytes) public static long maxBytesForLevel(int level, int levelFanoutSize, long maxSSTableSizeInBytes) { if (level == 0) - return 4L * maxSSTableSizeInBytes; + return MAX_SSTABLES_L0 * maxSSTableSizeInBytes; double bytes = Math.pow(levelFanoutSize, level) * maxSSTableSizeInBytes; if (bytes > Long.MAX_VALUE) throw new RuntimeException("At most " + Long.MAX_VALUE + " bytes may be in a compaction level; your maxSSTableSize must be absurdly high to compute " + bytes); @@ -188,17 +222,17 @@ public static long maxBytesForLevel(int level, int levelFanoutSize, long maxSSTa * @return highest-priority sstables to compact, and level to compact them to * If no compactions are necessary, will return null */ - public synchronized CompactionCandidate getCompactionCandidates() + synchronized CompactionAggregate.Leveled getCompactionCandidate() { // during bootstrap we only do size tiering in L0 to make sure // the streamed files can be placed in their original levels if (StorageService.instance.isBootstrapMode()) { - List mostInteresting = getSSTablesForSTCS(generations.get(0)); + CompactionPick mostInteresting = getSSTablesForSTCS(generations.get(0)); if (!mostInteresting.isEmpty()) { logger.info("Bootstrapping - doing STCS in L0"); - return new CompactionCandidate(mostInteresting, 0, Long.MAX_VALUE); + return getSTCSAggregate(mostInteresting); } return null; } @@ -232,7 +266,7 @@ public synchronized CompactionCandidate getCompactionCandidates() // Let's check that L0 is far enough behind to warrant STCS. // If it is, it will be used before proceeding any of higher level - CompactionCandidate l0Compaction = getSTCSInL0CompactionCandidate(); + CompactionAggregate.Leveled l0Compactions = getSTCSInL0CompactionCandidate(); for (int i = generations.levelCount() - 1; i > 0; i--) { @@ -241,7 +275,7 @@ public synchronized CompactionCandidate getCompactionCandidates() continue; // mostly this just avoids polluting the debug log with zero scores // we want to calculate score excluding compacting ones Set sstablesInLevel = Sets.newHashSet(sstables); - Set remaining = Sets.difference(sstablesInLevel, cfs.getTracker().getCompacting()); + Set remaining = Sets.difference(sstablesInLevel, dataTracker.getCompacting()); long remainingBytesForLevel = SSTableReader.getTotalBytes(remaining); long maxBytesForLevel = maxBytesForLevel(i, maxSSTableSizeInBytes); double score = (double) remainingBytesForLevel / (double) maxBytesForLevel; @@ -259,18 +293,20 @@ public synchronized CompactionCandidate getCompactionCandidates() } // before proceeding with a higher level, let's see if L0 is far enough behind to warrant STCS - if (l0Compaction != null) - return l0Compaction; + if (l0Compactions != null) + return l0Compactions; // L0 is fine, proceed with this level Collection candidates = getCandidatesFor(i); + int pendingCompactions = Math.max(0, getEstimatedPendingTasks(i) - 1); + if (!candidates.isEmpty()) { int nextLevel = getNextLevel(candidates); candidates = getOverlappingStarvedSSTables(nextLevel, candidates); if (logger.isTraceEnabled()) logger.trace("Compaction candidates for L{} are {}", i, toString(candidates)); - return new CompactionCandidate(candidates, nextLevel, maxSSTableSizeInBytes); + return CompactionAggregate.createLeveled(sstablesInLevel, candidates, pendingCompactions, maxSSTableSizeInBytes, i, nextLevel, score, levelFanoutSize); } else { @@ -280,7 +316,9 @@ public synchronized CompactionCandidate getCompactionCandidates() } // Higher levels are happy, time for a standard, non-STCS L0 compaction - if (generations.get(0).isEmpty()) + Set sstables = getLevel(0); + + if (sstables.isEmpty()) return null; Collection candidates = getCandidatesFor(0); if (candidates.isEmpty()) @@ -288,36 +326,51 @@ public synchronized CompactionCandidate getCompactionCandidates() // Since we don't have any other compactions to do, see if there is a STCS compaction to perform in L0; if // there is a long running compaction, we want to make sure that we continue to keep the number of SSTables // small in L0. - return l0Compaction; + return l0Compactions; } - return new CompactionCandidate(candidates, getNextLevel(candidates), maxSSTableSizeInBytes); + double l0Score = (double) SSTableReader.getTotalBytes(sstables) / (double) maxBytesForLevel(0, maxSSTableSizeInBytes); + int l0PendingCompactions = Math.max(0, getEstimatedPendingTasks(0) - 1); + return CompactionAggregate.createLeveled(sstables, candidates, l0PendingCompactions, maxSSTableSizeInBytes, 0, getNextLevel(candidates), l0Score, levelFanoutSize); } - private CompactionCandidate getSTCSInL0CompactionCandidate() + private CompactionAggregate.Leveled getSTCSInL0CompactionCandidate() { if (!DatabaseDescriptor.getDisableSTCSInL0() && generations.get(0).size() > MAX_COMPACTING_L0) { - List mostInteresting = getSSTablesForSTCS(generations.get(0)); + CompactionPick mostInteresting = getSSTablesForSTCS(getLevel(0)); if (!mostInteresting.isEmpty()) { logger.debug("L0 is too far behind, performing size-tiering there first"); - return new CompactionCandidate(mostInteresting, 0, Long.MAX_VALUE); + return getSTCSAggregate(mostInteresting); } } return null; } - private List getSSTablesForSTCS(Collection sstables) + private CompactionAggregate.Leveled getSTCSAggregate(CompactionPick compaction) + { + Set sstables = getLevel(0); + double score = (double) SSTableReader.getTotalBytes(sstables) / (double) maxBytesForLevel(0, maxSSTableSizeInBytes); + int remainingSSTables = sstables.size() - compaction.sstables.size(); + int pendingTasks = remainingSSTables > cfs.getMinimumCompactionThreshold() + ? (int) Math.ceil(remainingSSTables / cfs.getMaximumCompactionThreshold()) + : 0; + return CompactionAggregate.createLeveledForSTCS(sstables, compaction, pendingTasks, score, levelFanoutSize); + } + + private CompactionPick getSSTablesForSTCS(Collection sstables) { - Iterable candidates = cfs.getTracker().getUncompacting(sstables); - List> pairs = SizeTieredCompactionStrategy.createSSTableAndLengthPairs(AbstractCompactionStrategy.filterSuspectSSTables(candidates)); - List> buckets = SizeTieredCompactionStrategy.getBuckets(pairs, - options.bucketHigh, - options.bucketLow, - options.minSSTableSize); - return SizeTieredCompactionStrategy.mostInterestingBucket(buckets, - cfs.getMinimumCompactionThreshold(), cfs.getMaximumCompactionThreshold()); + Iterable candidates = dataTracker.getNoncompacting(sstables); + + SizeTieredCompactionStrategy.SizeTieredBuckets sizeTieredBuckets; + sizeTieredBuckets = new SizeTieredCompactionStrategy.SizeTieredBuckets(candidates, + options, + cfs.getMinimumCompactionThreshold(), + cfs.getMaximumCompactionThreshold()); + sizeTieredBuckets.aggregate(); + + return CompactionAggregate.getSelected(sizeTieredBuckets.getAggregates()); } /** @@ -358,10 +411,10 @@ private Collection getOverlappingStarvedSSTables(int targetLevel, PartitionPosition min = null; for (SSTableReader candidate : candidates) { - if (min == null || candidate.first.compareTo(min) < 0) - min = candidate.first; - if (max == null || candidate.last.compareTo(max) > 0) - max = candidate.last; + if (min == null || candidate.getFirst().compareTo(min) < 0) + min = candidate.getFirst(); + if (max == null || candidate.getLast().compareTo(max) > 0) + max = candidate.getLast(); } if (min == null || max == null || min.equals(max)) // single partition sstables - we cannot include a high level sstable. return candidates; @@ -369,7 +422,7 @@ private Collection getOverlappingStarvedSSTables(int targetLevel, Range boundaries = new Range<>(min, max); for (SSTableReader sstable : generations.get(i)) { - Range r = new Range<>(sstable.first, sstable.last); + Range r = new Range<>(sstable.getFirst(), sstable.getLast()); if (boundaries.contains(r) && !compacting.contains(sstable)) { logger.info("Adding high-level (L{}) {} to candidates", sstable.getSSTableLevel(), sstable); @@ -390,6 +443,14 @@ public synchronized int getLevelSize(int i) return generations.get(i).size(); } + public synchronized int[] getSSTableCountPerLevel() + { + int[] counts = new int[getLevelCount()]; + for (int i = 0; i < counts.length; i++) + counts[i] = getLevel(i).size(); + return counts; + } + public synchronized int[] getAllLevelSize() { return generations.getAllLevelSize(); @@ -425,20 +486,20 @@ private static Set overlapping(Collection candidat */ Iterator iter = candidates.iterator(); SSTableReader sstable = iter.next(); - Token first = sstable.first.getToken(); - Token last = sstable.last.getToken(); + Token first = sstable.getFirst().getToken(); + Token last = sstable.getLast().getToken(); while (iter.hasNext()) { sstable = iter.next(); - first = first.compareTo(sstable.first.getToken()) <= 0 ? first : sstable.first.getToken(); - last = last.compareTo(sstable.last.getToken()) >= 0 ? last : sstable.last.getToken(); + first = first.compareTo(sstable.getFirst().getToken()) <= 0 ? first : sstable.getFirst().getToken(); + last = last.compareTo(sstable.getLast().getToken()) >= 0 ? last : sstable.getLast().getToken(); } return overlapping(first, last, others); } - private static Set overlappingWithBounds(SSTableReader sstable, Map> others) + static Set overlappingWithBounds(SSTableReader sstable, Map> others) { - return overlappingWithBounds(sstable.first.getToken(), sstable.last.getToken(), others); + return overlappingWithBounds(sstable.getFirst().getToken(), sstable.getLast().getToken(), others); } /** @@ -464,27 +525,34 @@ private static Set overlappingWithBounds(Token start, Token end, return overlapped; } - private static Map> genBounds(Iterable ssTableReaders) + @VisibleForTesting + static Map> genBounds(Iterable ssTableReaders) { Map> boundsMap = new HashMap<>(); for (SSTableReader sstable : ssTableReaders) { - boundsMap.put(sstable, new Bounds<>(sstable.first.getToken(), sstable.last.getToken())); + boundsMap.put(sstable, new Bounds<>(sstable.getFirst().getToken(), sstable.getLast().getToken())); } return boundsMap; } /** + * Determine the highest-priority sstables to compact for the given level and add any overlapping sstables + * from the next level. + *

    * @return highest-priority sstables to compact for the given level. * If no compactions are possible (because of concurrent compactions or because some sstables are excluded * for prior failure), will return an empty list. Never returns null. + * + * @param level the level number + * @return highest-priority sstables to compact for the given level. */ private Collection getCandidatesFor(int level) { assert !generations.get(level).isEmpty(); logger.trace("Choosing candidates for L{}", level); - final Set compacting = cfs.getTracker().getCompacting(); + final Set compacting = dataTracker.getCompacting(); if (level == 0) { @@ -494,10 +562,10 @@ private Collection getCandidatesFor(int level) PartitionPosition firstCompactingKey = null; for (SSTableReader candidate : compactingL0) { - if (firstCompactingKey == null || candidate.first.compareTo(firstCompactingKey) < 0) - firstCompactingKey = candidate.first; - if (lastCompactingKey == null || candidate.last.compareTo(lastCompactingKey) > 0) - lastCompactingKey = candidate.last; + if (firstCompactingKey == null || candidate.getFirst().compareTo(firstCompactingKey) < 0) + firstCompactingKey = candidate.getFirst(); + if (lastCompactingKey == null || candidate.getLast().compareTo(lastCompactingKey) > 0) + lastCompactingKey = candidate.getLast(); } // L0 is the dumping ground for new sstables which thus may overlap each other. @@ -582,7 +650,7 @@ private Set getCompactingL0() { Set sstables = new HashSet<>(); Set levelSSTables = new HashSet<>(generations.get(0)); - for (SSTableReader sstable : cfs.getTracker().getCompacting()) + for (SSTableReader sstable : dataTracker.getCompacting()) { if (levelSSTables.contains(sstable)) sstables.add(sstable); @@ -617,32 +685,68 @@ public synchronized int getLevelCount() return 0; } - public synchronized int getEstimatedTasks() + public synchronized List getEstimatedTasks(CompactionAggregate.Leveled selected) { - long tasks = 0; - long[] estimated = new long[generations.levelCount()]; + List ret = new ArrayList<>(generations.levelCount()); for (int i = generations.levelCount() - 1; i >= 0; i--) { Set sstables = generations.get(i); - // If there is 1 byte over TBL - (MBL * 1.001), there is still a task left, so we need to round up. - estimated[i] = (long)Math.ceil((double)Math.max(0L, SSTableReader.getTotalBytes(sstables) - (long)(maxBytesForLevel(i, maxSSTableSizeInBytes) * 1.001)) / (double)maxSSTableSizeInBytes); - tasks += estimated[i]; - } - if (!DatabaseDescriptor.getDisableSTCSInL0() && generations.get(0).size() > cfs.getMaximumCompactionThreshold()) - { - int l0compactions = generations.get(0).size() / cfs.getMaximumCompactionThreshold(); - tasks += l0compactions; - estimated[0] += l0compactions; + // do not log high levels that are empty, only log after we've found a non-empty level + if (sstables.isEmpty() && ret.isEmpty()) + continue; + + if (selected != null && selected.level == i) + { + ret.add(selected); + continue; // pending tasks already calculated by getCompactionCandidate() + } + + if (i == 0) + { // for L0 if it is too far behind then pick the STCS choice + CompactionAggregate l0Compactions = getSTCSInL0CompactionCandidate(); + if (l0Compactions != null) + { + ret.add(l0Compactions); + continue; + } + } + + int pendingTasks = getEstimatedPendingTasks(i); + double score = (double) SSTableReader.getTotalBytes(sstables) / (double) maxBytesForLevel(i, maxSSTableSizeInBytes); + ret.add(CompactionAggregate.createLeveled(sstables, pendingTasks, maxSSTableSizeInBytes, i, score, levelFanoutSize)); } - logger.trace("Estimating {} compactions to do for {}.{}", - Arrays.toString(estimated), cfs.keyspace.getName(), cfs.name); - return Ints.checkedCast(tasks); + logger.trace("Estimating {} compactions to do for {}", ret.size(), cfs.metadata()); + return ret; } - public int getNextLevel(Collection sstables) + /** + * @return the estimated number of LCS compactions for a given level with the given sstables. Because it compacts one sstable at + * a time, this number is determined as the number of bytes above the maximum divided the maximum sstable size in bytes. + * + * This is however incorrect for L0. If the STCS threshold has been exceeded, we simply divide by the max threshold, + * otherwise we currently use a very pessimistic estimate (no overlapping sstables). + */ + private int getEstimatedPendingTasks(int level) + { + final Set sstables = getLevel(level); + if (sstables.isEmpty()) + return 0; + + final Set compacting = dataTracker.getCompacting(); + final Set remaining = Sets.difference(Sets.newHashSet(sstables), compacting); + + if (level == 0 && !DatabaseDescriptor.getDisableSTCSInL0() && remaining.size() > MAX_COMPACTING_L0) + return remaining.size() / cfs.getMaximumCompactionThreshold(); + + // If there is 1 byte over TBL - (MBL * 1.001), there is still a task left, so we need to round up. + return Math.toIntExact((long) Math.ceil((Math.max(0L, SSTableReader.getTotalBytes(remaining) - + (maxBytesForLevel(level, maxSSTableSizeInBytes) * 1.001)) / (double) maxSSTableSizeInBytes))); + } + + int getNextLevel(Collection sstables) { int maximumLevel = Integer.MIN_VALUE; int minimumLevel = Integer.MAX_VALUE; @@ -680,18 +784,4 @@ synchronized void newLevel(SSTableReader sstable, int oldLevel) generations.newLevel(sstable, oldLevel); lastCompactedSSTables[oldLevel] = sstable; } - - public static class CompactionCandidate - { - public final Collection sstables; - public final int level; - public final long maxSSTableBytes; - - public CompactionCandidate(Collection sstables, int level, long maxSSTableBytes) - { - this.sstables = sstables; - this.level = level; - this.maxSSTableBytes = maxSSTableBytes; - } - } } diff --git a/src/java/org/apache/cassandra/db/compaction/OperationType.java b/src/java/org/apache/cassandra/db/compaction/OperationType.java index e957e42c9df5..9429d2eaddec 100644 --- a/src/java/org/apache/cassandra/db/compaction/OperationType.java +++ b/src/java/org/apache/cassandra/db/compaction/OperationType.java @@ -17,6 +17,16 @@ */ package org.apache.cassandra.db.compaction; +import com.google.common.base.Predicate; + +/** + * The types of operations that can be observed with {@link AbstractTableOperation} and tracked by + * {@link org.apache.cassandra.db.lifecycle.LifecycleTransaction}. + *

    + * Historically these operations have been broadly described as "compactions", even though they have + * nothing to do with actual compactions. Any operation that can report progress and that normally + * involves files, either for reading or writing, is a valid operation. + */ public enum OperationType { /** Each modification here should be also applied to {@link org.apache.cassandra.tools.nodetool.Stop#compactionType} */ @@ -64,4 +74,9 @@ public String toString() { return type; } + + public static final Predicate REWRITES_SSTABLES = o -> o == COMPACTION || o == CLEANUP || o == SCRUB || + o == TOMBSTONE_COMPACTION || o == ANTICOMPACTION || + o == UPGRADE_SSTABLES || o == RELOCATE || + o == GARBAGE_COLLECT; } diff --git a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java index 03d411174503..07c1983f203c 100644 --- a/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java +++ b/src/java/org/apache/cassandra/db/compaction/PendingRepairHolder.java @@ -25,12 +25,12 @@ import java.util.UUID; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.index.Index; @@ -47,9 +47,9 @@ public class PendingRepairHolder extends AbstractStrategyHolder private final List managers = new ArrayList<>(); private final boolean isTransient; - public PendingRepairHolder(ColumnFamilyStore cfs, DestinationRouter router, boolean isTransient) + public PendingRepairHolder(ColumnFamilyStore cfs, CompactionStrategyFactory strategyFactory, DestinationRouter router, boolean isTransient) { - super(cfs, router); + super(cfs, strategyFactory, router); this.isTransient = isTransient; } @@ -70,7 +70,7 @@ public void setStrategyInternal(CompactionParams params, int numTokenPartitions) { managers.clear(); for (int i = 0; i < numTokenPartitions; i++) - managers.add(new PendingRepairManager(cfs, params, isTransient)); + managers.add(new PendingRepairManager(cfs, strategyFactory, params, isTransient)); } @Override @@ -82,24 +82,24 @@ public boolean managesRepairedGroup(boolean isRepaired, boolean isPendingRepair, } @Override - public AbstractCompactionStrategy getStrategyFor(SSTableReader sstable) + public LegacyAbstractCompactionStrategy getStrategyFor(SSTableReader sstable) { Preconditions.checkArgument(managesSSTable(sstable), "Attempting to get compaction strategy from wrong holder"); return managers.get(router.getIndexForSSTable(sstable)).getOrCreate(sstable); } @Override - public Iterable allStrategies() + public Iterable allStrategies() { return Iterables.concat(Iterables.transform(managers, PendingRepairManager::getStrategies)); } - Iterable getStrategiesFor(UUID session) + Iterable getStrategiesFor(UUID session) { - List strategies = new ArrayList<>(managers.size()); + List strategies = new ArrayList<>(managers.size()); for (PendingRepairManager manager : managers) { - AbstractCompactionStrategy strategy = manager.get(session); + LegacyAbstractCompactionStrategy strategy = manager.get(session); if (strategy != null) strategies.add(strategy); } @@ -112,11 +112,11 @@ public Iterable getManagers() } @Override - public Collection getBackgroundTaskSuppliers(int gcBefore) + public Collection getBackgroundTaskSuppliers(int gcBefore) { - List suppliers = new ArrayList<>(managers.size()); + List suppliers = new ArrayList<>(managers.size()); for (PendingRepairManager manager : managers) - suppliers.add(new TaskSupplier(manager.getMaxEstimatedRemainingTasks(), () -> manager.getNextBackgroundTask(gcBefore))); + suppliers.add(new TasksSupplier(manager.getMaxEstimatedRemainingTasks(), () -> manager.getNextBackgroundTasks(gcBefore))); return suppliers; } @@ -127,9 +127,7 @@ public Collection getMaximalTasks(int gcBefore, boolean List tasks = new ArrayList<>(managers.size()); for (PendingRepairManager manager : managers) { - Collection task = manager.getMaximalTasks(gcBefore, splitOutput); - if (task != null) - tasks.addAll(task); + tasks.addAll(manager.getMaximalTasks(gcBefore, splitOutput)); } return tasks; } @@ -149,31 +147,31 @@ public Collection getUserDefinedTasks(GroupedSSTableCont return tasks; } - AbstractCompactionTask getNextRepairFinishedTask() + Collection getNextRepairFinishedTasks() { - List repairFinishedSuppliers = getRepairFinishedTaskSuppliers(); + List repairFinishedSuppliers = getRepairFinishedTaskSuppliers(); if (!repairFinishedSuppliers.isEmpty()) { Collections.sort(repairFinishedSuppliers); - for (TaskSupplier supplier : repairFinishedSuppliers) + for (TasksSupplier supplier : repairFinishedSuppliers) { - AbstractCompactionTask task = supplier.getTask(); - if (task != null) - return task; + Collection tasks = supplier.getTasks(); + if (!tasks.isEmpty()) + return tasks; } } - return null; + return ImmutableList.of(); } - private ArrayList getRepairFinishedTaskSuppliers() + private ArrayList getRepairFinishedTaskSuppliers() { - ArrayList suppliers = new ArrayList<>(managers.size()); + ArrayList suppliers = new ArrayList<>(managers.size()); for (PendingRepairManager manager : managers) { int numPending = manager.getNumPendingRepairFinishedTasks(); if (numPending > 0) { - suppliers.add(new TaskSupplier(numPending, manager::getNextRepairFinishedTask)); + suppliers.add(new TasksSupplier(numPending, manager::getNextRepairFinishedTasks)); } } @@ -241,7 +239,7 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, boolean isTransient, MetadataCollector collector, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { Preconditions.checkArgument(repairedAt == ActiveRepairService.UNREPAIRED_SSTABLE, @@ -249,7 +247,7 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, Preconditions.checkArgument(pendingRepair != null, "PendingRepairHolder can't create sstable writer without pendingRepair id"); // to avoid creating a compaction strategy for the wrong pending repair manager, we get the index based on where the sstable is to be written - AbstractCompactionStrategy strategy = managers.get(router.getIndexForSSTableDirectory(descriptor)).getOrCreate(pendingRepair); + CompactionStrategy strategy = managers.get(router.getIndexForSSTableDirectory(descriptor)).getOrCreate(pendingRepair); return strategy.createSSTableMultiWriter(descriptor, keyCount, repairedAt, @@ -257,21 +255,10 @@ public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, isTransient, collector, header, - indexes, + indexGroups, lifecycleNewTracker); } - @Override - public int getStrategyIndex(AbstractCompactionStrategy strategy) - { - for (int i = 0; i < managers.size(); i++) - { - if (managers.get(i).hasStrategy(strategy)) - return i; - } - return -1; - } - public boolean hasDataForSession(UUID sessionID) { return Iterables.any(managers, prm -> prm.hasDataForSession(sessionID)); @@ -282,4 +269,9 @@ public boolean containsSSTable(SSTableReader sstable) { return Iterables.any(managers, prm -> prm.containsSSTable(sstable)); } + + public int size() + { + return managers.size(); + } } diff --git a/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java b/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java index aefa40be807e..7d49a581b439 100644 --- a/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java +++ b/src/java/org/apache/cassandra/db/compaction/PendingRepairManager.java @@ -29,30 +29,27 @@ import java.util.UUID; import java.util.stream.Collectors; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.repair.consistent.admin.CleanupSummary; import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.utils.Pair; /** - * Companion to CompactionStrategyManager which manages the sstables marked pending repair. + * This class manages the sstables marked pending repair so that they can be assigned to legacy compaction + * strategies via the legacy strategy container or manager. * * SSTables are classified as pending repair by the anti-compaction performed at the beginning * of an incremental repair, or when they're streamed in with a pending repair id. This prevents @@ -64,9 +61,10 @@ class PendingRepairManager private static final Logger logger = LoggerFactory.getLogger(PendingRepairManager.class); private final ColumnFamilyStore cfs; + private final CompactionStrategyFactory strategyFactory; private final CompactionParams params; private final boolean isTransient; - private volatile ImmutableMap strategies = ImmutableMap.of(); + private volatile ImmutableMap strategies = ImmutableMap.of(); /** * Indicates we're being asked to do something with an sstable that isn't marked pending repair @@ -79,34 +77,35 @@ public IllegalSSTableArgumentException(String s) } } - PendingRepairManager(ColumnFamilyStore cfs, CompactionParams params, boolean isTransient) + PendingRepairManager(ColumnFamilyStore cfs, CompactionStrategyFactory strategyFactory, CompactionParams params, boolean isTransient) { this.cfs = cfs; + this.strategyFactory = strategyFactory; this.params = params; this.isTransient = isTransient; } - private ImmutableMap.Builder mapBuilder() + private ImmutableMap.Builder mapBuilder() { return ImmutableMap.builder(); } - AbstractCompactionStrategy get(UUID id) + LegacyAbstractCompactionStrategy get(UUID id) { return strategies.get(id); } - AbstractCompactionStrategy get(SSTableReader sstable) + LegacyAbstractCompactionStrategy get(SSTableReader sstable) { assert sstable.isPendingRepair(); - return get(sstable.getSSTableMetadata().pendingRepair); + return get(sstable.getPendingRepair()); } - AbstractCompactionStrategy getOrCreate(UUID id) + LegacyAbstractCompactionStrategy getOrCreate(UUID id) { checkPendingID(id); assert id != null; - AbstractCompactionStrategy strategy = get(id); + LegacyAbstractCompactionStrategy strategy = get(id); if (strategy == null) { synchronized (this) @@ -116,7 +115,7 @@ AbstractCompactionStrategy getOrCreate(UUID id) if (strategy == null) { logger.debug("Creating {}.{} compaction strategy for pending repair: {}", cfs.metadata.keyspace, cfs.metadata.name, id); - strategy = cfs.createCompactionStrategyInstance(params); + strategy = strategyFactory.createLegacyStrategy(params); strategies = mapBuilder().putAll(strategies).put(id, strategy).build(); } } @@ -132,12 +131,12 @@ private static void checkPendingID(UUID pendingID) } } - AbstractCompactionStrategy getOrCreate(SSTableReader sstable) + LegacyAbstractCompactionStrategy getOrCreate(SSTableReader sstable) { - return getOrCreate(sstable.getSSTableMetadata().pendingRepair); + return getOrCreate(sstable.getPendingRepair()); } - private synchronized void removeSessionIfEmpty(UUID sessionID) + synchronized void removeSessionIfEmpty(UUID sessionID) { if (!strategies.containsKey(sessionID) || !strategies.get(sessionID).getSSTables().isEmpty()) return; @@ -148,14 +147,13 @@ private synchronized void removeSessionIfEmpty(UUID sessionID) synchronized void removeSSTable(SSTableReader sstable) { - for (Map.Entry entry : strategies.entrySet()) + for (Map.Entry entry : strategies.entrySet()) { entry.getValue().removeSSTable(sstable); removeSessionIfEmpty(entry.getKey()); } } - void removeSSTables(Iterable removed) { for (SSTableReader sstable : removed) @@ -183,7 +181,7 @@ synchronized void replaceSSTables(Set removed, Set Map, Set>> groups = new HashMap<>(); for (SSTableReader sstable : removed) { - UUID sessionID = sstable.getSSTableMetadata().pendingRepair; + UUID sessionID = sstable.getPendingRepair(); if (!groups.containsKey(sessionID)) { groups.put(sessionID, Pair.create(new HashSet<>(), new HashSet<>())); @@ -193,7 +191,7 @@ synchronized void replaceSSTables(Set removed, Set for (SSTableReader sstable : added) { - UUID sessionID = sstable.getSSTableMetadata().pendingRepair; + UUID sessionID = sstable.getPendingRepair(); if (!groups.containsKey(sessionID)) { groups.put(sessionID, Pair.create(new HashSet<>(), new HashSet<>())); @@ -203,7 +201,7 @@ synchronized void replaceSSTables(Set removed, Set for (Map.Entry, Set>> entry : groups.entrySet()) { - AbstractCompactionStrategy strategy = getOrCreate(entry.getKey()); + LegacyAbstractCompactionStrategy strategy = getOrCreate(entry.getKey()); Set groupRemoved = entry.getValue().left; Set groupAdded = entry.getValue().right; @@ -218,15 +216,15 @@ synchronized void replaceSSTables(Set removed, Set synchronized void startup() { - strategies.values().forEach(AbstractCompactionStrategy::startup); + strategies.values().forEach(CompactionStrategy::startup); } synchronized void shutdown() { - strategies.values().forEach(AbstractCompactionStrategy::shutdown); + strategies.values().forEach(CompactionStrategy::shutdown); } - private int getEstimatedRemainingTasks(UUID sessionID, AbstractCompactionStrategy strategy) + private int getEstimatedRemainingTasks(UUID sessionID, CompactionStrategy strategy) { if (canCleanup(sessionID)) { @@ -241,7 +239,7 @@ private int getEstimatedRemainingTasks(UUID sessionID, AbstractCompactionStrateg int getEstimatedRemainingTasks() { int tasks = 0; - for (Map.Entry entry : strategies.entrySet()) + for (Map.Entry entry : strategies.entrySet()) { tasks += getEstimatedRemainingTasks(entry.getKey(), entry.getValue()); } @@ -254,7 +252,7 @@ int getEstimatedRemainingTasks() int getMaxEstimatedRemainingTasks() { int tasks = 0; - for (Map.Entry entry : strategies.entrySet()) + for (Map.Entry entry : strategies.entrySet()) { tasks = Math.max(tasks, getEstimatedRemainingTasks(entry.getKey(), entry.getValue())); } @@ -265,63 +263,13 @@ int getMaxEstimatedRemainingTasks() private RepairFinishedCompactionTask getRepairFinishedCompactionTask(UUID sessionID) { Preconditions.checkState(canCleanup(sessionID)); - AbstractCompactionStrategy compactionStrategy = get(sessionID); + LegacyAbstractCompactionStrategy compactionStrategy = get(sessionID); if (compactionStrategy == null) return null; Set sstables = compactionStrategy.getSSTables(); long repairedAt = ActiveRepairService.instance.consistent.local.getFinalSessionRepairedAt(sessionID); LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); - return txn == null ? null : new RepairFinishedCompactionTask(cfs, txn, sessionID, repairedAt); - } - - public static class CleanupTask - { - private final ColumnFamilyStore cfs; - private final List> tasks; - - public CleanupTask(ColumnFamilyStore cfs, List> tasks) - { - this.cfs = cfs; - this.tasks = tasks; - } - - public CleanupSummary cleanup() - { - Set successful = new HashSet<>(); - Set unsuccessful = new HashSet<>(); - for (Pair pair : tasks) - { - UUID session = pair.left; - RepairFinishedCompactionTask task = pair.right; - - if (task != null) - { - try - { - task.run(); - successful.add(session); - } - catch (Throwable t) - { - t = task.transaction.abort(t); - logger.error("Failed cleaning up " + session, t); - unsuccessful.add(session); - } - } - else - { - unsuccessful.add(session); - } - } - return new CleanupSummary(cfs, successful, unsuccessful); - } - - public Throwable abort(Throwable accumulate) - { - for (Pair pair : tasks) - accumulate = pair.right.transaction.abort(accumulate); - return accumulate; - } + return txn == null ? null : new RepairFinishedCompactionTask(cfs, txn, sessionID, repairedAt, isTransient); } public CleanupTask releaseSessionData(Collection sessionIDs) @@ -350,26 +298,30 @@ synchronized int getNumPendingRepairFinishedTasks() return count; } - synchronized AbstractCompactionTask getNextRepairFinishedTask() + synchronized Collection getNextRepairFinishedTasks() { for (UUID sessionID : strategies.keySet()) { if (canCleanup(sessionID)) { - return getRepairFinishedCompactionTask(sessionID); + RepairFinishedCompactionTask task = getRepairFinishedCompactionTask(sessionID); + if (task != null) + return ImmutableList.of(task); + else + return ImmutableList.of(); } } - return null; + return ImmutableList.of(); } - synchronized AbstractCompactionTask getNextBackgroundTask(int gcBefore) + synchronized Collection getNextBackgroundTasks(int gcBefore) { if (strategies.isEmpty()) - return null; + return ImmutableList.of(); Map numTasks = new HashMap<>(strategies.size()); ArrayList sessions = new ArrayList<>(strategies.size()); - for (Map.Entry entry : strategies.entrySet()) + for (Map.Entry entry : strategies.entrySet()) { if (canCleanup(entry.getKey())) { @@ -380,22 +332,22 @@ synchronized AbstractCompactionTask getNextBackgroundTask(int gcBefore) } if (sessions.isEmpty()) - return null; + return ImmutableList.of(); // we want the session with the most compactions at the head of the list sessions.sort((o1, o2) -> numTasks.get(o2) - numTasks.get(o1)); UUID sessionID = sessions.get(0); - return get(sessionID).getNextBackgroundTask(gcBefore); + return get(sessionID).getNextBackgroundTasks(gcBefore); } synchronized Collection getMaximalTasks(int gcBefore, boolean splitOutput) { if (strategies.isEmpty()) - return null; + return ImmutableList.of(); List maximalTasks = new ArrayList<>(strategies.size()); - for (Map.Entry entry : strategies.entrySet()) + for (Map.Entry entry : strategies.entrySet()) { if (canCleanup(entry.getKey())) { @@ -403,15 +355,13 @@ synchronized Collection getMaximalTasks(int gcBefore, bo } else { - Collection tasks = entry.getValue().getMaximalTask(gcBefore, splitOutput); - if (tasks != null) - maximalTasks.addAll(tasks); + maximalTasks.addAll(entry.getValue().getMaximalTasks(gcBefore, splitOutput)); } } - return !maximalTasks.isEmpty() ? maximalTasks : null; + return maximalTasks; } - Collection getStrategies() + Collection getStrategies() { return strategies.values(); } @@ -437,7 +387,7 @@ synchronized Set getScanners(Collection sstables Map> sessionSSTables = new HashMap<>(); for (SSTableReader sstable : sstables) { - UUID sessionID = sstable.getSSTableMetadata().pendingRepair; + UUID sessionID = sstable.getPendingRepair(); checkPendingID(sessionID); sessionSSTables.computeIfAbsent(sessionID, k -> new HashSet<>()).add(sstable); } @@ -457,14 +407,14 @@ synchronized Set getScanners(Collection sstables return scanners; } - public boolean hasStrategy(AbstractCompactionStrategy strategy) + public boolean hasStrategy(CompactionStrategy strategy) { return strategies.values().contains(strategy); } public synchronized boolean hasDataForSession(UUID sessionID) { - return strategies.keySet().contains(sessionID); + return strategies.containsKey(sessionID); } boolean containsSSTable(SSTableReader sstable) @@ -479,79 +429,6 @@ boolean containsSSTable(SSTableReader sstable) public Collection createUserDefinedTasks(Collection sstables, int gcBefore) { Map> group = sstables.stream().collect(Collectors.groupingBy(s -> s.getSSTableMetadata().pendingRepair)); - return group.entrySet().stream().map(g -> strategies.get(g.getKey()).getUserDefinedTask(g.getValue(), gcBefore)).collect(Collectors.toList()); - } - - /** - * promotes/demotes sstables involved in a consistent repair that has been finalized, or failed - */ - class RepairFinishedCompactionTask extends AbstractCompactionTask - { - private final UUID sessionID; - private final long repairedAt; - - RepairFinishedCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction, UUID sessionID, long repairedAt) - { - super(cfs, transaction); - this.sessionID = sessionID; - this.repairedAt = repairedAt; - } - - @VisibleForTesting - UUID getSessionID() - { - return sessionID; - } - - protected void runMayThrow() throws Exception - { - boolean completed = false; - boolean obsoleteSSTables = isTransient && repairedAt > 0; - try - { - if (obsoleteSSTables) - { - logger.info("Obsoleting transient repaired sstables for {}", sessionID); - Preconditions.checkState(Iterables.all(transaction.originals(), SSTableReader::isTransient)); - transaction.obsoleteOriginals(); - } - else - { - logger.info("Moving {} from pending to repaired with repaired at = {} and session id = {}", transaction.originals(), repairedAt, sessionID); - cfs.getCompactionStrategyManager().mutateRepaired(transaction.originals(), repairedAt, ActiveRepairService.NO_PENDING_REPAIR, false); - } - completed = true; - } - finally - { - if (obsoleteSSTables) - { - transaction.finish(); - } - else - { - // we abort here because mutating metadata isn't guarded by LifecycleTransaction, so this won't roll - // anything back. Also, we don't want to obsolete the originals. We're only using it to prevent other - // compactions from marking these sstables compacting, and unmarking them when we're done - transaction.abort(); - } - if (completed) - { - removeSessionIfEmpty(sessionID); - } - } - } - - public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set nonExpiredSSTables) - { - throw new UnsupportedOperationException(); - } - - protected int executeInternal(ActiveCompactionsTracker activeCompactions) - { - run(); - return transaction.originals().size(); - } + return group.entrySet().stream().map(g -> strategies.get(g.getKey()).getUserDefinedTasks(g.getValue(), gcBefore)).flatMap(Collection::stream).collect(Collectors.toList()); } - } diff --git a/src/java/org/apache/cassandra/db/compaction/RepairFinishedCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/RepairFinishedCompactionTask.java new file mode 100644 index 000000000000..24769206b782 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/RepairFinishedCompactionTask.java @@ -0,0 +1,122 @@ +/* + * 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.compaction; + +import java.util.Set; +import java.util.UUID; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.service.ActiveRepairService; + +/** + * promotes/demotes sstables involved in a consistent repair that has been finalized, or failed + */ +public class RepairFinishedCompactionTask extends AbstractCompactionTask +{ + private static final Logger logger = LoggerFactory.getLogger(RepairFinishedCompactionTask.class); + + private final UUID sessionID; + private final long repairedAt; + private final boolean isTransient; + + public RepairFinishedCompactionTask(ColumnFamilyStore cfs, + LifecycleTransaction transaction, + UUID sessionID, + long repairedAt, + boolean isTransient) + { + super(cfs, transaction); + this.sessionID = sessionID; + this.repairedAt = repairedAt; + this.isTransient = isTransient; + } + + @VisibleForTesting + UUID getSessionID() + { + return sessionID; + } + + protected void runMayThrow() throws Exception + { + boolean completed = false; + boolean obsoleteSSTables = isTransient && repairedAt > 0; + try + { + if (obsoleteSSTables) + { + logger.info("Obsoleting transient repaired sstables for {}", sessionID); + Preconditions.checkState(Iterables.all(transaction.originals(), SSTableReader::isTransient)); + transaction.obsoleteOriginals(); + } + else + { + logger.info("Moving {} from pending to repaired with repaired at = {} and session id = {}", transaction.originals(), repairedAt, sessionID); + CompactionStrategyContainer compactionStrategyContainer = cfs.getCompactionStrategyContainer(); + cfs.mutateRepaired(compactionStrategyContainer.getWriteLock(), + transaction.originals(), + repairedAt, + ActiveRepairService.NO_PENDING_REPAIR, + false); + compactionStrategyContainer.repairSessionCompleted(sessionID); + } + completed = true; + } + finally + { + if (obsoleteSSTables) + { + transaction.finish(); + } + else + { + // we abort here because mutating metadata isn't guarded by LifecycleTransaction, so this won't roll + // anything back. Also, we don't want to obsolete the originals. We're only using it to prevent other + // compactions from marking these sstables compacting, and unmarking them when we're done + transaction.abort(); + } + if (completed) + { + cfs.getCompactionStrategyContainer().repairSessionCompleted(sessionID); + } + } + } + + public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set nonExpiredSSTables) + { + throw new UnsupportedOperationException(); + } + + protected int executeInternal() + { + run(); + return transaction.originals().size(); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java index 1746d7c2abde..c4b7332da879 100644 --- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java +++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java @@ -19,7 +19,6 @@ import java.util.*; import java.util.function.LongPredicate; -import java.util.function.Predicate; import org.apache.cassandra.db.*; import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; @@ -29,7 +28,7 @@ public class SSTableSplitter { - private final SplittingCompactionTask task; + private final AbstractCompactionTask task; public SSTableSplitter(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB) { @@ -38,16 +37,16 @@ public SSTableSplitter(ColumnFamilyStore cfs, LifecycleTransaction transaction, public void split() { - task.execute(ActiveCompactionsTracker.NOOP); + task.execute(); } - public static class SplittingCompactionTask extends CompactionTask + private static class SplittingCompactionTask extends CompactionTask { private final int sstableSizeInMB; public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB) { - super(cfs, transaction, CompactionManager.NO_GC, false); + super(cfs, transaction, CompactionManager.NO_GC, false, null); this.sstableSizeInMB = sstableSizeInMB; if (sstableSizeInMB <= 0) diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java index afbfe3d27a61..0d84b9d13619 100644 --- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java +++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java @@ -25,9 +25,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; +import org.apache.cassandra.io.sstable.format.ScrubPartitionIterator; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.*; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; @@ -60,19 +60,13 @@ public class Scrubber implements Closeable private final ReadWriteLock fileAccessLock; private final RandomAccessReader dataFile; - private final RandomAccessReader indexFile; + private ScrubPartitionIterator indexIterator; private final ScrubInfo scrubInfo; - private final RowIndexEntry.IndexSerializer rowIndexEntrySerializer; private int goodRows; private int badRows; private int emptyRows; - private ByteBuffer currentIndexKey; - private ByteBuffer nextIndexKey; - long currentRowPositionFromIndex; - long nextRowPositionFromIndex; - private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics(); private final OutputHandler outputHandler; @@ -111,21 +105,18 @@ public Scrubber(ColumnFamilyStore cfs, this.outputHandler = outputHandler; this.skipCorrupted = skipCorrupted; this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows; - this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(cfs.metadata(), - sstable.descriptor.version, - sstable.header); List toScrub = Collections.singletonList(sstable); this.destination = cfs.getDirectories().getLocationForDisk(cfs.getDiskBoundaries().getCorrectDiskForSSTable(sstable)); this.isCommutative = cfs.metadata().isCounter(); - boolean hasIndexFile = (new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))).exists(); + boolean hasIndexFile = sstable.hasIndex(); this.isIndex = cfs.isIndex(); if (!hasIndexFile) { // if there's any corruption in the -Data.db then rows can't be skipped over. but it's worth a shot. - outputHandler.warn("Missing component: " + sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)); + outputHandler.warn("Missing index component"); } this.checkData = checkData && !this.isIndex; //LocalByPartitionerType does not support validation this.expectedBloomFilterSize = Math.max( @@ -141,19 +132,36 @@ public Scrubber(ColumnFamilyStore cfs, ? sstable.openDataReader() : sstable.openDataReader(CompactionManager.instance.getRateLimiter()); - this.indexFile = hasIndexFile - ? RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))) - : null; + try + { + this.indexIterator = hasIndexFile + ? openIndexIterator() + : null; + } + catch (RuntimeException ex) + { + outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex); + } this.scrubInfo = new ScrubInfo(dataFile, sstable, fileAccessLock.readLock()); - this.currentRowPositionFromIndex = 0; - this.nextRowPositionFromIndex = 0; - if (reinsertOverflowedTTLRows) outputHandler.output("Starting scrub with reinsert overflowed TTL option"); } + private ScrubPartitionIterator openIndexIterator() + { + try + { + return sstable.scrubPartitionsIterator(); + } + catch (IOException e) + { + outputHandler.warn("Index is unreadable."); + } + return null; + } + private UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename) { return checkData ? UnfilteredRowIterators.withValidation(iter, filename) : iter; @@ -167,13 +175,7 @@ public void scrub() try (SSTableRewriter writer = SSTableRewriter.construct(cfs, transaction, false, sstable.maxDataAge); Refs refs = Refs.ref(Collections.singleton(sstable))) { - nextIndexKey = indexAvailable() ? ByteBufferUtil.readWithShortLength(indexFile) : null; - if (indexAvailable()) - { - // throw away variable so we don't have a side effect in the assert - long firstRowPositionFromIndex = rowIndexEntrySerializer.deserializePositionAndSkip(indexFile); - assert firstRowPositionFromIndex == 0 : firstRowPositionFromIndex; - } + assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition(); StatsMetadata metadata = sstable.getSSTableMetadata(); writer.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, metadata.repairedAt, metadata.pendingRepair, metadata.isTransient, sstable, transaction)); @@ -183,44 +185,66 @@ public void scrub() while (!dataFile.isEOF()) { if (scrubInfo.isStopRequested()) - throw new CompactionInterruptedException(scrubInfo.getCompactionInfo()); + throw new CompactionInterruptedException(scrubInfo.getProgress()); - long rowStart = dataFile.getFilePointer(); - outputHandler.debug("Reading row at " + rowStart); + // position in a data file where the partition starts + long dataStart = dataFile.getFilePointer(); + outputHandler.debug("Reading row at " + dataStart); DecoratedKey key = null; + Throwable keyReadError = null; try { key = sstable.decorateKey(ByteBufferUtil.readWithShortLength(dataFile)); } catch (Throwable th) { + keyReadError = th; throwIfFatal(th); // check for null key below } - updateIndexKey(); - - long dataStart = dataFile.getFilePointer(); - + // position of the partition in a data file, it points to the beginning of the partition key long dataStartFromIndex = -1; + // size of the partition (including partition key) long dataSizeFromIndex = -1; - if (currentIndexKey != null) + ByteBuffer currentIndexKey = null; + if (indexAvailable()) { - dataStartFromIndex = currentRowPositionFromIndex + 2 + currentIndexKey.remaining(); - dataSizeFromIndex = nextRowPositionFromIndex - dataStartFromIndex; + currentIndexKey = indexIterator.key(); + dataStartFromIndex = indexIterator.dataPosition(); + if (!indexIterator.isExhausted()) + { + try + { + indexIterator.advance(); + if (!indexIterator.isExhausted()) + dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex; + } + catch (Throwable th) + { + throwIfFatal(th); + outputHandler.warn(String.format( + "Failed to advance to the next index position. Index is corrupted. " + + "Continuing without the index. " + + "Last position read is %d.", indexIterator.dataPosition()), th); + indexIterator.close(); + indexIterator = null; + currentIndexKey = null; + dataStartFromIndex = -1; + dataSizeFromIndex = -1; + } + } } // avoid an NPE if key is null String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.getKey()); outputHandler.debug(String.format("row %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSizeFromIndex))); - assert currentIndexKey != null || !indexAvailable(); - try { if (key == null) - throw new IOError(new IOException("Unable to read row key from data file")); + throw new IOError(new IOException("Unable to read row key from data file", keyReadError)); if (currentIndexKey != null && !key.getKey().equals(currentIndexKey)) { @@ -229,10 +253,10 @@ public void scrub() "_too big_", ByteBufferUtil.bytesToHex(currentIndexKey)))); } - if (indexFile != null && dataSizeFromIndex > dataFile.length()) + if (indexIterator != null && dataSizeFromIndex > dataFile.length()) throw new IOError(new IOException("Impossible row size (greater than file length): " + dataSizeFromIndex)); - if (indexFile != null && dataStart != dataStartFromIndex) + if (indexIterator != null && dataStart != dataStartFromIndex) outputHandler.warn(String.format("Data file row position %d differs from index file row position %d", dataStart, dataStartFromIndex)); if (tryAppend(prevKey, key, writer)) @@ -246,12 +270,14 @@ public void scrub() if (currentIndexKey != null && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex)) { + // position where the row should start in a data file (right after the partition key) + long rowStartFromIndex = dataStartFromIndex + TypeSizes.SHORT_SIZE + currentIndexKey.remaining(); outputHandler.output(String.format("Retrying from row index; data is %s bytes starting at %s", - dataSizeFromIndex, dataStartFromIndex)); + dataSizeFromIndex, rowStartFromIndex)); key = sstable.decorateKey(currentIndexKey); try { - dataFile.seek(dataStartFromIndex); + dataFile.seek(rowStartFromIndex); if (tryAppend(prevKey, key, writer)) prevKey = key; @@ -270,10 +296,23 @@ public void scrub() { throwIfCannotContinue(key, th); - outputHandler.warn("Row starting at position " + dataStart + " is unreadable; skipping to next"); badRows++; - if (currentIndexKey != null) + if (indexIterator != null) + { + outputHandler.warn("Row starting at position " + dataStart + " is unreadable; skipping to next"); seekToNextRow(); + } + else + { + outputHandler.warn(String.format( + "Unrecoverable error while scrubbing %s." + + "Scrubbing cannot continue. The sstable will be marked for deletion. " + + "You can attempt manual recovery from the pre-scrub snapshot. " + + "You can also run nodetool repair to transfer the data from a healthy replica, if any.", + sstable)); + // There's no way to resync and continue. Give up. + break; + } } } } @@ -295,13 +334,10 @@ public void scrub() } // finish obsoletes the old sstable + transaction.obsoleteOriginals(); finished.addAll(writer.setRepairedAt(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt).finish()); completed = true; } - catch (IOException e) - { - throw Throwables.propagate(e); - } finally { if (transaction.isOffline()) @@ -370,36 +406,17 @@ private UnfilteredRowIterator getIterator(DecoratedKey key) negativeLocalDeletionInfoMetrics) : rowMergingIterator; } - private void updateIndexKey() - { - currentIndexKey = nextIndexKey; - currentRowPositionFromIndex = nextRowPositionFromIndex; - try - { - nextIndexKey = !indexAvailable() ? null : ByteBufferUtil.readWithShortLength(indexFile); - - nextRowPositionFromIndex = !indexAvailable() - ? dataFile.length() - : rowIndexEntrySerializer.deserializePositionAndSkip(indexFile); - } - catch (Throwable th) - { - JVMStabilityInspector.inspectThrowable(th); - outputHandler.warn("Error reading index file", th); - nextIndexKey = null; - nextRowPositionFromIndex = dataFile.length(); - } - } - private boolean indexAvailable() { - return indexFile != null && !indexFile.isEOF(); + return indexIterator != null && !indexIterator.isExhausted(); } private void seekToNextRow() { - while(nextRowPositionFromIndex < dataFile.length()) + while (!indexIterator.isExhausted()) { + long nextRowPositionFromIndex = indexIterator.dataPosition(); + try { dataFile.seek(nextRowPositionFromIndex); @@ -412,7 +429,15 @@ private void seekToNextRow() badRows++; } - updateIndexKey(); + try + { + indexIterator.advance(); + } + catch (Throwable th) + { + outputHandler.warn("Failed to go to the next entry in index", th); + throw Throwables.cleaned(th); + } } } @@ -455,7 +480,7 @@ public void close() try { FileUtils.closeQuietly(dataFile); - FileUtils.closeQuietly(indexFile); + FileUtils.closeQuietly(indexIterator); } finally { @@ -463,12 +488,12 @@ public void close() } } - public CompactionInfo.Holder getScrubInfo() + public TableOperation getScrubInfo() { return scrubInfo; } - private static class ScrubInfo extends CompactionInfo.Holder + private static class ScrubInfo extends AbstractTableOperation { private final RandomAccessReader dataFile; private final SSTableReader sstable; @@ -483,17 +508,17 @@ public ScrubInfo(RandomAccessReader dataFile, SSTableReader sstable, Lock fileRe scrubCompactionId = UUIDGen.getTimeUUID(); } - public CompactionInfo getCompactionInfo() + public OperationProgress getProgress() { fileReadLock.lock(); try { - return new CompactionInfo(sstable.metadata(), - OperationType.SCRUB, - dataFile.getFilePointer(), - dataFile.length(), - scrubCompactionId, - ImmutableSet.of(sstable)); + return new OperationProgress(sstable.metadata(), + OperationType.SCRUB, + dataFile.getFilePointer(), + dataFile.length(), + scrubCompactionId, + ImmutableSet.of(sstable)); } catch (Exception e) { diff --git a/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java b/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java index 2e1dffc5221d..3958fba09d75 100644 --- a/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java +++ b/src/java/org/apache/cassandra/db/compaction/SingleSSTableLCSTask.java @@ -43,11 +43,12 @@ public class SingleSSTableLCSTask extends AbstractCompactionTask private final int level; - public SingleSSTableLCSTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int level) + public SingleSSTableLCSTask(LeveledCompactionStrategy strategy, LifecycleTransaction txn, int level) { - super(cfs, txn); + super(strategy.cfs, txn); assert txn.originals().size() == 1; this.level = level; + addObserver(strategy); } @Override @@ -56,8 +57,13 @@ public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Dir throw new UnsupportedOperationException("This method should never be called on SingleSSTableLCSTask"); } + int getLevel() + { + return level; + } + @Override - protected int executeInternal(ActiveCompactionsTracker activeCompactions) + protected int executeInternal() { run(); return 1; diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStatistics.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStatistics.java new file mode 100644 index 000000000000..e5592ed48c9b --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStatistics.java @@ -0,0 +1,51 @@ +/* + * 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.compaction; + +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The statistics for size tiered compaction. + *

    + * Implements serializable to allow structured info to be returned via JMX. + */ +public class SizeTieredCompactionStatistics extends TieredCompactionStatistics +{ + /** The average sstable size in this tier */ + private final long avgSSTableSize; + + SizeTieredCompactionStatistics(CompactionAggregateStatistics base, long avgSSTableSize) + { + super(base); + this.avgSSTableSize = avgSSTableSize; + } + + /** The average sstable size in this tier */ + public long avgSSTableSize() + { + return avgSSTableSize; + } + + @Override + @JsonProperty("Bucket") + protected String tierValue() + { + return toString(avgSSTableSize); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java index 8d1d8dac2eb3..6a90e2889dc4 100644 --- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java @@ -18,9 +18,14 @@ package org.apache.cassandra.db.compaction; import java.util.*; -import java.util.Map.Entry; +import java.util.stream.Collectors; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import org.slf4j.Logger; @@ -36,275 +41,301 @@ import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.utils.Pair; -import static com.google.common.collect.Iterables.filter; - -public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy +public class SizeTieredCompactionStrategy extends LegacyAbstractCompactionStrategy.WithAggregates { private static final Logger logger = LoggerFactory.getLogger(SizeTieredCompactionStrategy.class); - private static final Comparator,Double>> bucketsByHotnessComparator = new Comparator, Double>>() - { - public int compare(Pair, Double> o1, Pair, Double> o2) - { - int comparison = Double.compare(o1.right, o2.right); - if (comparison != 0) - return comparison; - - // break ties by compacting the smallest sstables first (this will probably only happen for - // system tables and new/unread sstables) - return Long.compare(avgSize(o1.left), avgSize(o2.left)); - } - - private long avgSize(List sstables) - { - long n = 0; - for (SSTableReader sstable : sstables) - n += sstable.bytesOnDisk(); - return n / sstables.size(); - } - }; + /** + * Compare {@link CompactionPick} instances by hotness first and in case of a tie by sstable size by + * selecting the largest first (a tie would happen for system tables and new/unread sstables). + *

    + * Note that in previous version there is a comment saying "break ties by compacting the smallest sstables first" + * but the code was doing the opposite. I preserved the behavior and fixed the comment. + */ + private static final Comparator comparePicksByHotness = Comparator.comparing(CompactionPick::hotness) + .thenComparing(CompactionPick::avgSizeInBytes); protected SizeTieredCompactionStrategyOptions sizeTieredOptions; - protected volatile int estimatedRemainingTasks; @VisibleForTesting protected final Set sstables = new HashSet<>(); - public SizeTieredCompactionStrategy(ColumnFamilyStore cfs, Map options) + public SizeTieredCompactionStrategy(CompactionStrategyFactory factory, Map options) { - super(cfs, options); - this.estimatedRemainingTasks = 0; + super(factory, options); this.sizeTieredOptions = new SizeTieredCompactionStrategyOptions(options); } - private synchronized List getNextBackgroundSSTables(final int gcBefore) + @Override + protected synchronized CompactionAggregate getNextBackgroundAggregate(final int gcBefore) { // make local copies so they can't be changed out from under us mid-method int minThreshold = cfs.getMinimumCompactionThreshold(); int maxThreshold = cfs.getMaximumCompactionThreshold(); - Iterable candidates = filterSuspectSSTables(filter(cfs.getUncompactingSSTables(), sstables::contains)); + List candidates = new ArrayList<>(); + synchronized (sstables) + { + Iterables.addAll(candidates, nonSuspectAndNotIn(sstables, dataTracker.getCompacting())); + } + + SizeTieredBuckets sizeTieredBuckets = new SizeTieredBuckets(candidates, sizeTieredOptions, minThreshold, maxThreshold); + sizeTieredBuckets.aggregate(); - List> buckets = getBuckets(createSSTableAndLengthPairs(candidates), sizeTieredOptions.bucketHigh, sizeTieredOptions.bucketLow, sizeTieredOptions.minSSTableSize); - logger.trace("Compaction buckets are {}", buckets); - estimatedRemainingTasks = getEstimatedCompactionsByTasks(cfs, buckets); - cfs.getCompactionStrategyManager().compactionLogger.pending(this, estimatedRemainingTasks); - List mostInteresting = mostInterestingBucket(buckets, minThreshold, maxThreshold); - if (!mostInteresting.isEmpty()) - return mostInteresting; + backgroundCompactions.setPending(this, sizeTieredBuckets.getAggregates()); + + CompactionAggregate ret = sizeTieredBuckets.getAggregates().isEmpty() ? null : sizeTieredBuckets.getAggregates().get(0); // if there is no sstable to compact in standard way, try compacting single sstable whose droppable tombstone // ratio is greater than threshold. - List sstablesWithTombstones = new ArrayList<>(); - for (SSTableReader sstable : candidates) - { - if (worthDroppingTombstones(sstable, gcBefore)) - sstablesWithTombstones.add(sstable); - } - if (sstablesWithTombstones.isEmpty()) - return Collections.emptyList(); + if (ret == null || ret.isEmpty()) + ret = makeTombstoneCompaction(gcBefore, candidates, list -> Collections.max(list, SSTableReader.sizeComparator)); - return Collections.singletonList(Collections.max(sstablesWithTombstones, SSTableReader.sizeComparator)); + return ret; } - /** - * @param buckets list of buckets from which to return the most interesting, where "interesting" is the total hotness for reads - * @param minThreshold minimum number of sstables in a bucket to qualify as interesting - * @param maxThreshold maximum number of sstables to compact at once (the returned bucket will be trimmed down to this) - * @return a bucket (list) of sstables to compact + * This class contains the logic for {@link SizeTieredCompactionStrategy}: + * + * - sorts the sstables by length on disk + * - it sorts the candidates into buckets + * - takes a snapshot of the sstable hotness + * - it organizes the buckets into a list of {@link CompactionAggregate}, an aggregate per bucket. + * An aggregate will have a list of compaction picks, each pick is a list of sstables below the max threshold, + * sorted by hotness. + * - the aggregates are sorted by comparing the total hotness of the first pick of each aggregate + * - the aggregate with the hottest first pick will have its first pick submitted for compaction. */ - public static List mostInterestingBucket(List> buckets, int minThreshold, int maxThreshold) + @NotThreadSafe + final static class SizeTieredBuckets { - // skip buckets containing less than minThreshold sstables, and limit other buckets to maxThreshold sstables - final List, Double>> prunedBucketsAndHotness = new ArrayList<>(buckets.size()); - for (List bucket : buckets) + private final SizeTieredCompactionStrategyOptions options; + private final List tablesBySize; + private final Map> buckets; + private final Map hotnessSnapshot; + private final int minThreshold; + private final int maxThreshold; + + /** + * This is the list of compactions order by most interesting first + */ + private List aggregates; + + /** + * @param candidates list sstables that are not yet compacting + * @param options the options for size tiered compaction strategy + * @param minThreshold minimum number of sstables in a bucket to qualify as interesting + * @param maxThreshold maximum number of sstables to compact at once (the returned bucket will be trimmed down to this) + */ + SizeTieredBuckets(Iterable candidates, + SizeTieredCompactionStrategyOptions options, + int minThreshold, + int maxThreshold) { - Pair, Double> bucketAndHotness = trimToThresholdWithHotness(bucket, maxThreshold); - if (bucketAndHotness != null && bucketAndHotness.left.size() >= minThreshold) - prunedBucketsAndHotness.add(bucketAndHotness); + this.options = options; + this.tablesBySize = new ArrayList<>(); + Iterables.addAll(this.tablesBySize, candidates); + this.tablesBySize.sort(SSTableReader.sizeComparator); + this.buckets = getBuckets(tablesBySize, options); + this.hotnessSnapshot = getHotnessSnapshot(buckets.values()); + this.minThreshold = minThreshold; + this.maxThreshold = maxThreshold; + + this.aggregates = new ArrayList<>(buckets.size()); + + if (logger.isTraceEnabled()) + logger.trace("Compaction buckets are {}", buckets); } - if (prunedBucketsAndHotness.isEmpty()) - return Collections.emptyList(); - - Pair, Double> hottest = Collections.max(prunedBucketsAndHotness, bucketsByHotnessComparator); - return hottest.left; - } - /** - * Returns a (bucket, hotness) pair or null if there were not enough sstables in the bucket to meet minThreshold. - * If there are more than maxThreshold sstables, the coldest sstables will be trimmed to meet the threshold. - **/ - @VisibleForTesting - static Pair, Double> trimToThresholdWithHotness(List bucket, int maxThreshold) - { - // Sort by sstable hotness (descending). We first build a map because the hotness may change during the sort. - final Map hotnessSnapshot = getHotnessMap(bucket); - Collections.sort(bucket, new Comparator() + /** + * Group sstables of similar on disk size into buckets. + * The given set must be sorted using SSTableReader.sizeComparator + */ + private static Map> getBuckets(List sstables, SizeTieredCompactionStrategyOptions options) { - public int compare(SSTableReader o1, SSTableReader o2) + if (sstables.isEmpty()) + return Collections.EMPTY_MAP; + + Map> buckets = new HashMap<>(); + + long currentAverageSize = 0; + List currentBucket = new ArrayList<>(); + + for (SSTableReader sstable: sstables) { - return -1 * Double.compare(hotnessSnapshot.get(o1), hotnessSnapshot.get(o2)); + long size = sstable.onDiskLength(); + assert size >= currentAverageSize; + + if (size >= currentAverageSize * options.bucketHigh + && size >= options.minSSTableSize + && currentAverageSize > 0) // false for first table only + { + // Switch to new bucket + buckets.put(currentAverageSize, currentBucket); + currentBucket = new ArrayList<>(); + } + // TODO: Is it okay that the bucket max can grow unboundedly? + + currentAverageSize = (currentAverageSize * currentBucket.size() + size) / (currentBucket.size() + 1); + currentBucket.add(sstable); } - }); - // and then trim the coldest sstables off the end to meet the maxThreshold - List prunedBucket = bucket.subList(0, Math.min(bucket.size(), maxThreshold)); + buckets.put(currentAverageSize, currentBucket); + return buckets; + } - // bucket hotness is the sum of the hotness of all sstable members - double bucketHotness = 0.0; - for (SSTableReader sstr : prunedBucket) - bucketHotness += hotness(sstr); + /** + * For each bucket with at least minThreshold sstables: + *

    + * - sort the sstables by hotness + * - divide the bucket into max threshold sstables and add it to a temporary list of candidates along with the total hotness of the bucket section + *

    + * Then select the candidate with the max hotness and the most interesting bucket and put the remaining candidates in the pending list. + * + * @return the parent object {@link SizeTieredBuckets} + */ + SizeTieredBuckets aggregate() + { + if (!aggregates.isEmpty()) + return this; // already called - return Pair.create(prunedBucket, bucketHotness); - } + List aggregatesWithoutCompactions = new ArrayList<>(buckets.size()); + List aggregatesWithCompactions = new ArrayList<>(buckets.size()); - private static Map getHotnessMap(Collection sstables) - { - Map hotness = new HashMap<>(sstables.size()); - for (SSTableReader sstable : sstables) - hotness.put(sstable, hotness(sstable)); - return hotness; - } + for (Map.Entry> entry : buckets.entrySet()) + { + long avgSizeBytes = entry.getKey(); + long minSizeBytes = (long) (avgSizeBytes * options.bucketLow); + long maxSizeBytes = (long) (avgSizeBytes * options.bucketHigh); - /** - * Returns the reads per second per key for this sstable, or 0.0 if the sstable has no read meter - */ - private static double hotness(SSTableReader sstr) - { - // system tables don't have read meters, just use 0.0 for the hotness - return sstr.getReadMeter() == null ? 0.0 : sstr.getReadMeter().twoHourRate() / sstr.estimatedKeys(); - } + List bucket = entry.getValue(); + double hotness = totHotness(bucket, hotnessSnapshot); - @SuppressWarnings("resource") - public AbstractCompactionTask getNextBackgroundTask(int gcBefore) - { - List previousCandidate = null; - while (true) - { - List hottestBucket = getNextBackgroundSSTables(gcBefore); + if (bucket.size() < minThreshold) + { + if (logger.isTraceEnabled()) + logger.trace("Aggregate with {} avg bytes for {} files not considered for compaction: {}", avgSizeBytes, bucket.size(), bucket); + + aggregatesWithoutCompactions.add(CompactionAggregate.createSizeTiered(bucket, + CompactionPick.EMPTY, + ImmutableList.of(), + hotness, + avgSizeBytes, + minSizeBytes, + maxSizeBytes)); + + continue; + } + + // sort the bucket by hotness + Collections.sort(bucket, (o1, o2) -> -1 * Double.compare(hotnessSnapshot.get(o1), hotnessSnapshot.get(o2))); + + // now divide the candidates into a list of picks, each pick with at most max threshold sstables + int i = 0; + CompactionPick selected = null; + List pending = new ArrayList<>(); + + + while ((bucket.size() - i) >= minThreshold) + { + List sstables = bucket.subList(i, i + Math.min(bucket.size() - i, maxThreshold)); + if (selected == null) + selected = CompactionPick.create(avgSizeBytes, sstables, totHotness(sstables, hotnessSnapshot)); + else + pending.add(CompactionPick.create(avgSizeBytes, sstables, totHotness(sstables, hotnessSnapshot))); + + i += sstables.size(); + } + + if (logger.isTraceEnabled()) + logger.trace("Aggregate with {} avg bytes for {} files considered for compaction: {}", avgSizeBytes, bucket.size(), bucket); - if (hottestBucket.isEmpty()) - return null; + // Finally create the new aggregate with the new pending compactions and those already compacting and not yet completed + aggregatesWithCompactions.add(CompactionAggregate.createSizeTiered(bucket, selected, pending, hotness, avgSizeBytes, minSizeBytes, maxSizeBytes)); + } + + // This sorts the aggregates based on the hotness of their selected pick so that the aggregate with the hottest selected pick + // be first in the list and get submitted + if (!aggregatesWithCompactions.isEmpty()) + { + Collections.sort(aggregatesWithCompactions, (a1, a2) -> comparePicksByHotness.compare(a2.getSelected(), a1.getSelected())); - // Already tried acquiring references without success. It means there is a race with - // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager - if (hottestBucket.equals(previousCandidate)) + if (logger.isTraceEnabled()) + logger.trace("Found compaction for aggregate {}", aggregatesWithCompactions.get(0)); + } + else { - logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + - "unless it happens frequently, in which case it must be reported. Will retry later.", - hottestBucket); - return null; + if (logger.isTraceEnabled()) + logger.trace("No compactions found"); } - LifecycleTransaction transaction = cfs.getTracker().tryModify(hottestBucket, OperationType.COMPACTION); - if (transaction != null) - return new CompactionTask(cfs, transaction, gcBefore); - previousCandidate = hottestBucket; + // publish the results + this.aggregates.addAll(aggregatesWithCompactions); // those with compactions first, because the first one will be the one submitted + this.aggregates.addAll(aggregatesWithoutCompactions); // then add those empty + return this; } - } - @SuppressWarnings("resource") - public synchronized Collection getMaximalTask(final int gcBefore, boolean splitOutput) - { - Iterable filteredSSTables = filterSuspectSSTables(sstables); - if (Iterables.isEmpty(filteredSSTables)) - return null; - LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION); - if (txn == null) - return null; - if (splitOutput) - return Arrays.asList(new SplittingCompactionTask(cfs, txn, gcBefore)); - return Arrays.asList(new CompactionTask(cfs, txn, gcBefore)); - } + /** + * For diagnostics only. Returns the sorted tables paired with their on-disk length. + */ + public Collection> pairs() + { + return Collections2.transform(tablesBySize, (SSTableReader table) -> Pair.create(table, table.onDiskLength())); + } - @SuppressWarnings("resource") - public AbstractCompactionTask getUserDefinedTask(Collection sstables, final int gcBefore) - { - assert !sstables.isEmpty(); // checked for by CM.submitUserDefined + public List> buckets() + { + return new ArrayList<>(buckets.values()); + } - LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); - if (transaction == null) + public List getAggregates() { - logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables); - return null; + return aggregates; } - return new CompactionTask(cfs, transaction, gcBefore).setUserDefined(true); + public List getCompactions() + { + return aggregates.stream().flatMap(aggr -> aggr.getActive().stream()).collect(Collectors.toList()); + } } - public int getEstimatedRemainingTasks() + /** + * @return a snapshot mapping sstables to their current read hotness. + */ + @VisibleForTesting + static Map getHotnessSnapshot(Collection> buckets) { - return estimatedRemainingTasks; - } + Map ret = new HashMap<>(); - public static List> createSSTableAndLengthPairs(Iterable sstables) - { - List> sstableLengthPairs = new ArrayList<>(Iterables.size(sstables)); - for(SSTableReader sstable : sstables) - sstableLengthPairs.add(Pair.create(sstable, sstable.onDiskLength())); - return sstableLengthPairs; + for (List sstables: buckets) + { + for (SSTableReader sstable : sstables) + ret.put(sstable, sstable.hotness()); + } + + return ret; } - /* - * Group files of similar size into buckets. + /** + * @return the sum of the hotness of all the sstables */ - public static List> getBuckets(Collection> files, double bucketHigh, double bucketLow, long minSSTableSize) + private static double totHotness(Iterable sstables, @Nullable final Map hotnessSnapshot) { - // Sort the list in order to get deterministic results during the grouping below - List> sortedFiles = new ArrayList>(files); - Collections.sort(sortedFiles, new Comparator>() - { - public int compare(Pair p1, Pair p2) - { - return p1.right.compareTo(p2.right); - } - }); - - Map> buckets = new HashMap>(); - - outer: - for (Pair pair: sortedFiles) + double hotness = 0.0; + for (SSTableReader sstable : sstables) { - long size = pair.right; - - // look for a bucket containing similar-sized files: - // group in the same bucket if it's w/in 50% of the average for this bucket, - // or this file and the bucket are all considered "small" (less than `minSSTableSize`) - for (Entry> entry : buckets.entrySet()) - { - List bucket = entry.getValue(); - long oldAverageSize = entry.getKey(); - if ((size > (oldAverageSize * bucketLow) && size < (oldAverageSize * bucketHigh)) - || (size < minSSTableSize && oldAverageSize < minSSTableSize)) - { - // remove and re-add under new new average size - buckets.remove(oldAverageSize); - long totalSize = bucket.size() * oldAverageSize; - long newAverageSize = (totalSize + size) / (bucket.size() + 1); - bucket.add(pair.left); - buckets.put(newAverageSize, bucket); - continue outer; - } - } - - // no similar bucket found; put it in a new one - ArrayList bucket = new ArrayList(); - bucket.add(pair.left); - buckets.put(size, bucket); + double h = hotnessSnapshot == null ? 0.0 : hotnessSnapshot.getOrDefault(sstable, 0.0); + hotness += h == 0.0 ? sstable.hotness() : h; } - return new ArrayList>(buckets.values()); + return hotness; } - public static int getEstimatedCompactionsByTasks(ColumnFamilyStore cfs, List> tasks) + @Override + protected AbstractCompactionTask createCompactionTask(final int gcBefore, LifecycleTransaction txn, boolean isMaximal, boolean splitOutput) { - int n = 0; - for (List bucket : tasks) - { - if (bucket.size() >= cfs.getMinimumCompactionThreshold()) - n += Math.ceil((double)bucket.size() / cfs.getMaximumCompactionThreshold()); - } - return n; + return isMaximal && splitOutput + ? new SplittingCompactionTask(cfs, txn, gcBefore, this) + : new CompactionTask(cfs, txn, gcBefore, false, this); } public long getMaxSSTableBytes() @@ -314,7 +345,7 @@ public long getMaxSSTableBytes() public static Map validateOptions(Map options) throws ConfigurationException { - Map uncheckedOptions = AbstractCompactionStrategy.validateOptions(options); + Map uncheckedOptions = CompactionStrategyOptions.validateOptions(options); uncheckedOptions = SizeTieredCompactionStrategyOptions.validateOptions(options, uncheckedOptions); uncheckedOptions.remove(CompactionParams.Option.MIN_THRESHOLD.toString()); @@ -324,21 +355,47 @@ public static Map validateOptions(Map options) t } @Override - public synchronized void addSSTable(SSTableReader added) + public void replaceSSTables(Collection removed, Collection added) + { + synchronized (sstables) + { + for (SSTableReader remove : removed) + sstables.remove(remove); + sstables.addAll(added); + } + } + + @Override + public void addSSTable(SSTableReader added) + { + synchronized (sstables) + { + sstables.add(added); + } + } + + @Override + void removeDeadSSTables() { - sstables.add(added); + removeDeadSSTables(sstables); } @Override - public synchronized void removeSSTable(SSTableReader sstable) + public void removeSSTable(SSTableReader sstable) { - sstables.remove(sstable); + synchronized (sstables) + { + sstables.remove(sstable); + } } @Override - protected Set getSSTables() + public Set getSSTables() { - return ImmutableSet.copyOf(sstables); + synchronized (sstables) + { + return ImmutableSet.copyOf(sstables); + } } public String toString() @@ -350,9 +407,9 @@ public String toString() private static class SplittingCompactionTask extends CompactionTask { - public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore) + public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, CompactionStrategy strategy) { - super(cfs, txn, gcBefore); + super(cfs, txn, gcBefore, false, strategy); } @Override diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java index 288af2bebb22..3b612eaedc31 100644 --- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java +++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java @@ -23,12 +23,12 @@ public final class SizeTieredCompactionStrategyOptions { - protected static final long DEFAULT_MIN_SSTABLE_SIZE = 50L * 1024L * 1024L; - protected static final double DEFAULT_BUCKET_LOW = 0.5; - protected static final double DEFAULT_BUCKET_HIGH = 1.5; - protected static final String MIN_SSTABLE_SIZE_KEY = "min_sstable_size"; - protected static final String BUCKET_LOW_KEY = "bucket_low"; - protected static final String BUCKET_HIGH_KEY = "bucket_high"; + static final long DEFAULT_MIN_SSTABLE_SIZE = 50L * 1024L * 1024L; + static final double DEFAULT_BUCKET_LOW = 0.5; + static final double DEFAULT_BUCKET_HIGH = 1.5; + static final String MIN_SSTABLE_SIZE_KEY = "min_sstable_size"; + static final String BUCKET_LOW_KEY = "bucket_low"; + static final String BUCKET_HIGH_KEY = "bucket_high"; protected long minSSTableSize; protected double bucketLow; @@ -46,9 +46,14 @@ public SizeTieredCompactionStrategyOptions(Map options) public SizeTieredCompactionStrategyOptions() { - minSSTableSize = DEFAULT_MIN_SSTABLE_SIZE; - bucketLow = DEFAULT_BUCKET_LOW; - bucketHigh = DEFAULT_BUCKET_HIGH; + this(DEFAULT_MIN_SSTABLE_SIZE, DEFAULT_BUCKET_LOW, DEFAULT_BUCKET_HIGH); + } + + SizeTieredCompactionStrategyOptions(long minSSTableSize, double bucketLow, double bucketHigh) + { + this.minSSTableSize = minSSTableSize; + this.bucketLow = bucketLow; + this.bucketHigh = bucketHigh; } private static double parseDouble(Map options, String key, double defaultValue) throws ConfigurationException diff --git a/src/java/org/apache/cassandra/db/compaction/TableOperation.java b/src/java/org/apache/cassandra/db/compaction/TableOperation.java new file mode 100644 index 000000000000..1908eeaa2c31 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/TableOperation.java @@ -0,0 +1,194 @@ +/* + * 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.compaction; + +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.function.Predicate; + +import javax.annotation.Nullable; + +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.TableMetadata; + +/** + * This is a table operation that must be able to report the operation progress and to + * interrupt the operation when requested. + *

    + * Any operation defined by {@link OperationType} is normally implementing this interface, + * for example index building, view building, cache saving, anti-compaction, compaction, + * scrubbing, verifying, tombstone collection and others. + *

    + * These operations have in common that they run on the compaction executor and used to be + * known as "compaction". + * */ +public interface TableOperation +{ + /** + * @return the progress of the operation, see {@link Progress}. + */ + AbstractTableOperation.OperationProgress getProgress(); + + /** + * Interrupt the operation. + */ + void stop(); + + /** + * Interrupt the current operation if possible and if the predicate is true. + * + * @param trigger cause of compaction interruption + */ + void stop(StopTrigger trigger); + + /** + * @return true if the operation has been requested to be interrupted. + */ + boolean isStopRequested(); + + /** + * Return true if the predicate for the given sstables holds, or if the operation + * does not consider any sstables, in which case it will always return true (the + * default behaviour). + *

    + * + * @param predicate the predicate to be applied to the operation sstables + * + * @return true by default, see overrides for different behaviors + */ + boolean shouldStop(Predicate predicate); + + /** + * @return cause of compaction interruption. + */ + public StopTrigger trigger(); + + /** + * if this compaction involves several/all tables we can safely check globalCompactionsPaused + * in isStopRequested() below + */ + public abstract boolean isGlobal(); + + /** + * The unit for the {@link Progress} report. + */ + enum Unit + { + BYTES("bytes"), RANGES("token range parts"), KEYS("keys"); + + private final String name; + + Unit(String name) + { + this.name = name; + } + + @Override + public String toString() + { + return this.name; + } + + public static boolean isFileSize(String unit) + { + return BYTES.toString().equals(unit); + } + } + + public enum StopTrigger + { + NONE(false), + TRUNCATE(true); + + private final boolean isFinal; + + StopTrigger(boolean isFinal) + { + this.isFinal = isFinal; + } + + // A stop trigger marked as final should not be overwritten. So a table operation that is + // marked with a final stop trigger cannot have it's stop trigger changed to another value. + public boolean isFinal() + { + return isFinal; + } + } + + /** + * The progress of a table operation. + */ + interface Progress + { + String ID = "id"; + String KEYSPACE = "keyspace"; + String COLUMNFAMILY = "columnfamily"; + String COMPLETED = "completed"; + String TOTAL = "total"; + String OPERATION_TYPE = "operationType"; + String UNIT = "unit"; + String OPERATION_ID = "operationId"; + + /** + * @return the keyspace name, if the metadata is not null. + */ + Optional keyspace(); + + /** + * @return the table name, if the metadata is not null. + */ + Optional table(); + + /** + * @return the table metadata, this may be null if the operation has no metadata. + */ + @Nullable TableMetadata metadata(); + + /** + * @return the number of units completed, see {@link this#unit()}. + */ + long completed(); + + /** + * @return the total number of units that must be processed by the operation, see {@link this#unit()}. + */ + long total(); + + /** + * @return the type of operation, see {@link OperationType}. + */ + OperationType operationType(); + + /** + * @return a unique identifier for this operation. + */ + UUID operationId(); + + /** + * @return the unit to be used for {@link this#completed()} and {@link this#total()}, see {@link Unit}. + */ + Unit unit(); + + /** + * @return a set of SSTables participating in this operation + */ + Set sstables(); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/TableOperationObserver.java b/src/java/org/apache/cassandra/db/compaction/TableOperationObserver.java new file mode 100644 index 000000000000..93dc643a5685 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/TableOperationObserver.java @@ -0,0 +1,41 @@ +/* + * 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.compaction; + +import org.apache.cassandra.utils.NonThrowingCloseable; + +/** + * An observer of {@link AbstractTableOperation}. + *

    + * The observer is notified when an operation is started. It returns a closeable that will be closed + * when the operation is finished. The operation can be queried at any time to get the progress information. + */ +public interface TableOperationObserver +{ + TableOperationObserver NOOP = operation -> () -> {}; + + /** + * Signal to the observer that an operation is starting. + * + * @param operation the operation starting + * + * @return a closeable that the caller should close when the operation completes + */ + NonThrowingCloseable onOperationStart(TableOperation operation); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/TieredCompactionStatistics.java b/src/java/org/apache/cassandra/db/compaction/TieredCompactionStatistics.java new file mode 100644 index 000000000000..f785a180efdf --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/TieredCompactionStatistics.java @@ -0,0 +1,68 @@ +/* + * 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.compaction; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; + +abstract class TieredCompactionStatistics extends CompactionAggregateStatistics +{ + private static final Collection HEADER = ImmutableList.copyOf(Iterables.concat(ImmutableList.of("Bucket", "Hotness"), + CompactionAggregateStatistics.HEADER)); + + private static final long serialVersionUID = 3695927592357987916L; + + public TieredCompactionStatistics(CompactionAggregateStatistics base) + { + super(base); + } + + /** The total read hotness of the sstables */ + @JsonProperty + public double hotness() + { + return hotness; + } + + @Override + protected Collection header() + { + return HEADER; + } + + @Override + protected Collection data() + { + List data = new ArrayList<>(HEADER.size()); + data.add(tierValue()); + data.add(String.format("%.4f", hotness)); + + data.addAll(super.data()); + + return data; + } + + protected abstract String tierValue(); +} diff --git a/src/java/org/apache/cassandra/db/compaction/TimeTieredCompactionStatistics.java b/src/java/org/apache/cassandra/db/compaction/TimeTieredCompactionStatistics.java new file mode 100644 index 000000000000..ce935e058932 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/TimeTieredCompactionStatistics.java @@ -0,0 +1,57 @@ +/* + * 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.compaction; + +import java.text.DateFormat; +import java.util.Date; + +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The statistics for time tiered compaction. + *

    + * Implements serializable to allow structured info to be returned via JMX. + */ +public class TimeTieredCompactionStatistics extends TieredCompactionStatistics +{ + protected static final DateFormat bucketFormatter = DateFormat.getDateTimeInstance(DateFormat.SHORT, DateFormat.SHORT); + + /** The timestamp in this tier */ + private final long timestamp; + + TimeTieredCompactionStatistics(CompactionAggregateStatistics base, long timestamp) + { + super(base); + + this.timestamp = timestamp; + } + + /** The timestamp in this tier */ + public long timestamp() + { + return timestamp; + } + + @Override + @JsonProperty("Bucket") + protected String tierValue() + { + return bucketFormatter.format(new Date(timestamp)); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java index 41668053ebe6..f6ec22cb18d2 100644 --- a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategy.java @@ -21,8 +21,9 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Iterator; -import java.util.TreeSet; +import java.util.Comparator; +import java.util.NavigableMap; +import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.HashSet; import java.util.List; @@ -34,34 +35,30 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.CompactionParams; -import org.apache.cassandra.utils.Pair; import static com.google.common.collect.Iterables.filter; -public class TimeWindowCompactionStrategy extends AbstractCompactionStrategy +public class TimeWindowCompactionStrategy extends LegacyAbstractCompactionStrategy.WithAggregates { private static final Logger logger = LoggerFactory.getLogger(TimeWindowCompactionStrategy.class); - private final TimeWindowCompactionStrategyOptions options; - protected volatile int estimatedRemainingTasks; + private final TimeWindowCompactionStrategyOptions twcsOptions; private final Set sstables = new HashSet<>(); private long lastExpiredCheck; private long highestWindowSeen; - public TimeWindowCompactionStrategy(ColumnFamilyStore cfs, Map options) + public TimeWindowCompactionStrategy(CompactionStrategyFactory factory, Map options) { - super(cfs, options); - this.estimatedRemainingTasks = 0; - this.options = new TimeWindowCompactionStrategyOptions(options); - if (!options.containsKey(AbstractCompactionStrategy.TOMBSTONE_COMPACTION_INTERVAL_OPTION) && !options.containsKey(AbstractCompactionStrategy.TOMBSTONE_THRESHOLD_OPTION)) + super(factory, options); + this.twcsOptions = new TimeWindowCompactionStrategyOptions(options); + if (!options.containsKey(CompactionStrategyOptions.TOMBSTONE_COMPACTION_INTERVAL_OPTION) && !options.containsKey(CompactionStrategyOptions.TOMBSTONE_THRESHOLD_OPTION)) { - disableTombstoneCompactions = true; + super.options.setDisableTombstoneCompactions(true); logger.debug("Disabling tombstone compactions for TWCS"); } else @@ -69,32 +66,12 @@ public TimeWindowCompactionStrategy(ColumnFamilyStore cfs, Map o } @Override - @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute - public AbstractCompactionTask getNextBackgroundTask(int gcBefore) + public AbstractCompactionTask createCompactionTask(final int gcBefore, + LifecycleTransaction txn, + boolean isMaximal, + boolean splitOutput) { - List previousCandidate = null; - while (true) - { - List latestBucket = getNextBackgroundSSTables(gcBefore); - - if (latestBucket.isEmpty()) - return null; - - // Already tried acquiring references without success. It means there is a race with - // the tracker but candidate SSTables were not yet replaced in the compaction strategy manager - if (latestBucket.equals(previousCandidate)) - { - logger.warn("Could not acquire references for compacting SSTables {} which is not a problem per se," + - "unless it happens frequently, in which case it must be reported. Will retry later.", - latestBucket); - return null; - } - - LifecycleTransaction modifier = cfs.getTracker().tryModify(latestBucket, OperationType.COMPACTION); - if (modifier != null) - return new TimeWindowCompactionTask(cfs, modifier, gcBefore, options.ignoreOverlaps); - previousCandidate = latestBucket; - } + return new TimeWindowCompactionTask(cfs, txn, gcBefore, ignoreOverlaps(), this); } /** @@ -102,21 +79,27 @@ public AbstractCompactionTask getNextBackgroundTask(int gcBefore) * @param gcBefore * @return */ - private synchronized List getNextBackgroundSSTables(final int gcBefore) + @Override + protected synchronized CompactionAggregate getNextBackgroundAggregate(final int gcBefore) { - if (Iterables.isEmpty(cfs.getSSTables(SSTableSet.LIVE))) - return Collections.emptyList(); + if (Iterables.isEmpty(dataTracker.getView().select(SSTableSet.LIVE))) + return null; - Set uncompacting = ImmutableSet.copyOf(filter(cfs.getUncompactingSSTables(), sstables::contains)); + Set compacting = dataTracker.getCompacting(); + Set uncompacting; + synchronized (sstables) + { + uncompacting = ImmutableSet.copyOf(filter(sstables, sstable -> !compacting.contains(sstable))); + } // Find fully expired SSTables. Those will be included no matter what. Set expired = Collections.emptySet(); - if (System.currentTimeMillis() - lastExpiredCheck > options.expiredSSTableCheckFrequency) + if (System.currentTimeMillis() - lastExpiredCheck > twcsOptions.expiredSSTableCheckFrequency) { logger.debug("TWCS expired check sufficiently far in the past, checking for fully expired SSTables"); - expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, options.ignoreOverlaps ? Collections.emptySet() : cfs.getOverlappingLiveSSTables(uncompacting), - gcBefore, options.ignoreOverlaps); + expired = CompactionController.getFullyExpiredSSTables(cfs, uncompacting, twcsOptions.ignoreOverlaps ? Collections.emptySet() : cfs.getOverlappingLiveSSTables(uncompacting), + gcBefore, twcsOptions.ignoreOverlaps); lastExpiredCheck = System.currentTimeMillis(); } else @@ -124,281 +107,272 @@ private synchronized List getNextBackgroundSSTables(final int gcB logger.debug("TWCS skipping check for fully expired SSTables"); } - Set candidates = Sets.newHashSet(filterSuspectSSTables(uncompacting)); + Set candidates = Sets.newHashSet(Iterables.filter(uncompacting, sstable -> !sstable.isMarkedSuspect())); + + CompactionAggregate compactionCandidate = getNextNonExpiredSSTables(Sets.difference(candidates, expired), gcBefore); + if (expired.isEmpty()) + return compactionCandidate; - List compactionCandidates = new ArrayList<>(getNextNonExpiredSSTables(Sets.difference(candidates, expired), gcBefore)); - if (!expired.isEmpty()) + logger.debug("Including expired sstables: {}", expired); + if (compactionCandidate == null) { - logger.debug("Including expired sstables: {}", expired); - compactionCandidates.addAll(expired); + long timestamp = getWindowBoundsInMillis(twcsOptions.sstableWindowUnit, twcsOptions.sstableWindowSize, + Collections.max(expired, Comparator.comparing(SSTableReader::getMaxTimestamp)).getMaxTimestamp()); + return CompactionAggregate.createTimeTiered(expired, timestamp); } - return compactionCandidates; + return compactionCandidate.withExpired(expired); } - private List getNextNonExpiredSSTables(Iterable nonExpiringSSTables, final int gcBefore) + private CompactionAggregate getNextNonExpiredSSTables(Iterable nonExpiringSSTables, final int gcBefore) { - List mostInteresting = getCompactionCandidates(nonExpiringSSTables); + List candidates = getCompactionCandidates(nonExpiringSSTables); + backgroundCompactions.setPending(this, candidates); - if (mostInteresting != null) - { - return mostInteresting; - } + CompactionAggregate ret = candidates.isEmpty() ? null : candidates.get(0); // if there is no sstable to compact in standard way, try compacting single sstable whose droppable tombstone // ratio is greater than threshold. - List sstablesWithTombstones = new ArrayList<>(); - for (SSTableReader sstable : nonExpiringSSTables) - { - if (worthDroppingTombstones(sstable, gcBefore)) - sstablesWithTombstones.add(sstable); - } - if (sstablesWithTombstones.isEmpty()) - return Collections.emptyList(); + if (ret == null || ret.isEmpty()) + ret = makeTombstoneCompaction(gcBefore, nonExpiringSSTables, list -> Collections.min(list, SSTableReader.sizeComparator)); - return Collections.singletonList(Collections.min(sstablesWithTombstones, SSTableReader.sizeComparator)); + return ret; } - private List getCompactionCandidates(Iterable candidateSSTables) + private List getCompactionCandidates(Iterable candidateSSTables) { - Pair, Long> buckets = getBuckets(candidateSSTables, options.sstableWindowUnit, options.sstableWindowSize, options.timestampResolution); + NavigableMap> buckets = getBuckets(candidateSSTables, twcsOptions.sstableWindowUnit, twcsOptions.sstableWindowSize, twcsOptions.timestampResolution); // Update the highest window seen, if necessary - if(buckets.right > this.highestWindowSeen) - this.highestWindowSeen = buckets.right; + if (!buckets.isEmpty()) + { + long maxKey = buckets.lastKey(); + if (maxKey > this.highestWindowSeen) + this.highestWindowSeen = maxKey; + } - NewestBucket mostInteresting = newestBucket(buckets.left, - cfs.getMinimumCompactionThreshold(), - cfs.getMaximumCompactionThreshold(), - options.stcsOptions, - this.highestWindowSeen); - - this.estimatedRemainingTasks = mostInteresting.estimatedRemainingTasks; - if (!mostInteresting.sstables.isEmpty()) - return mostInteresting.sstables; - return null; + return getBucketAggregates(buckets, + cfs.getMinimumCompactionThreshold(), + cfs.getMaximumCompactionThreshold(), + twcsOptions.stcsOptions, + this.highestWindowSeen); } + @Override - public synchronized void addSSTable(SSTableReader sstable) + public void replaceSSTables(Collection removed, Collection added) { - sstables.add(sstable); + synchronized (sstables) + { + for (SSTableReader remove : removed) + sstables.remove(remove); + sstables.addAll(added); + } } @Override - public synchronized void removeSSTable(SSTableReader sstable) + public void addSSTable(SSTableReader sstable) { - sstables.remove(sstable); + synchronized (sstables) + { + sstables.add(sstable); + } } @Override - protected Set getSSTables() + void removeDeadSSTables() { - return ImmutableSet.copyOf(sstables); + removeDeadSSTables(sstables); } - /** - * Find the lowest and highest timestamps in a given timestamp/unit pair - * Returns milliseconds, caller should adjust accordingly - */ - public static Pair getWindowBoundsInMillis(TimeUnit windowTimeUnit, int windowTimeSize, long timestampInMillis) + @Override + public void removeSSTable(SSTableReader sstable) { - long lowerTimestamp; - long upperTimestamp; - long timestampInSeconds = TimeUnit.SECONDS.convert(timestampInMillis, TimeUnit.MILLISECONDS); + synchronized (sstables) + { + sstables.remove(sstable); + } + } - switch(windowTimeUnit) + @Override + public Set getSSTables() + { + synchronized (sstables) { - case MINUTES: - lowerTimestamp = timestampInSeconds - ((timestampInSeconds) % (60L * windowTimeSize)); - upperTimestamp = (lowerTimestamp + (60L * (windowTimeSize - 1L))) + 59L; - break; - case HOURS: - lowerTimestamp = timestampInSeconds - ((timestampInSeconds) % (3600L * windowTimeSize)); - upperTimestamp = (lowerTimestamp + (3600L * (windowTimeSize - 1L))) + 3599L; - break; - case DAYS: - default: - lowerTimestamp = timestampInSeconds - ((timestampInSeconds) % (86400L * windowTimeSize)); - upperTimestamp = (lowerTimestamp + (86400L * (windowTimeSize - 1L))) + 86399L; - break; + return ImmutableSet.copyOf(sstables); } + } - return Pair.create(TimeUnit.MILLISECONDS.convert(lowerTimestamp, TimeUnit.SECONDS), - TimeUnit.MILLISECONDS.convert(upperTimestamp, TimeUnit.SECONDS)); + /** + * Find the lowest timestamp in a given window/unit pair and + * return it expressed as milliseconds, the caller should adjust accordingly + */ + static long getWindowBoundsInMillis(TimeUnit windowTimeUnit, int windowTimeSize, long timestampInMillis) + { + long sizeInMillis = TimeUnit.MILLISECONDS.convert(windowTimeSize, windowTimeUnit); + return (timestampInMillis / sizeInMillis) * sizeInMillis; } /** * Group files with similar max timestamp into buckets. + *

    + * The max timestamp of each sstable is converted into the timestamp resolution and then the window bounds are + * calculated by calling {@link #getWindowBoundsInMillis(TimeUnit, int, long)}. The sstable is added to the bucket + * with the same lower timestamp bound. If the lower timestamp bound is higher than any other seen, then it is recorded + * as the max timestamp seen that will be returned. * - * @param files pairs consisting of a file and its min timestamp - * @param sstableWindowUnit - * @param sstableWindowSize - * @param timestampResolution - * @return A pair, where the left element is the bucket representation (map of timestamp to sstablereader), and the right is the highest timestamp seen + * @param files the candidate sstables + * @param sstableWindowUnit the time unit for {@code sstableWindowSize} + * @param sstableWindowSize the size of the time window by which sstables are grouped + * @param timestampResolution the time unit for converting the sstable timestamp + * @return A pair, where the left element is the bucket representation (multi-map of lower bound timestamp to sstables), + * and the right is the highest lower bound timestamp seen */ @VisibleForTesting - static Pair, Long> getBuckets(Iterable files, TimeUnit sstableWindowUnit, int sstableWindowSize, TimeUnit timestampResolution) + static NavigableMap> getBuckets(Iterable files, TimeUnit sstableWindowUnit, int sstableWindowSize, TimeUnit timestampResolution) { - HashMultimap buckets = HashMultimap.create(); + NavigableMap> buckets = new TreeMap<>(Long::compare); - long maxTimestamp = 0; - // Create hash map to represent buckets // For each sstable, add sstable to the time bucket // Where the bucket is the file's max timestamp rounded to the nearest window bucket for (SSTableReader f : files) { assert TimeWindowCompactionStrategyOptions.validTimestampTimeUnits.contains(timestampResolution); long tStamp = TimeUnit.MILLISECONDS.convert(f.getMaxTimestamp(), timestampResolution); - Pair bounds = getWindowBoundsInMillis(sstableWindowUnit, sstableWindowSize, tStamp); - buckets.put(bounds.left, f); - if (bounds.left > maxTimestamp) - maxTimestamp = bounds.left; + addToBuckets(buckets, f, tStamp, sstableWindowUnit, sstableWindowSize); } - logger.trace("buckets {}, max timestamp {}", buckets, maxTimestamp); - return Pair.create(buckets, maxTimestamp); + logger.trace("buckets {}, max timestamp {}", buckets, buckets.isEmpty() ? "none" : buckets.lastKey().toString()); + return buckets; } - static final class NewestBucket + @VisibleForTesting + static void addToBuckets(NavigableMap> buckets, SSTableReader f, long tStamp, TimeUnit sstableWindowUnit, int sstableWindowSize) { - /** The sstables that should be compacted next */ - final List sstables; - - /** The number of tasks estimated */ - final int estimatedRemainingTasks; - - NewestBucket(List sstables, int estimatedRemainingTasks) - { - this.sstables = sstables; - this.estimatedRemainingTasks = estimatedRemainingTasks; - } - - @Override - public String toString() - { - return String.format("sstables: %s, estimated remaining tasks: %d", sstables, estimatedRemainingTasks); - } + long bound = getWindowBoundsInMillis(sstableWindowUnit, sstableWindowSize, tStamp); + buckets.computeIfAbsent(bound, + key -> new ArrayList<>()) + .add(f); } - /** - * @param buckets list of buckets, sorted from newest to oldest, from which to return the newest bucket within thresholds. + * If the current bucket has at least minThreshold SSTables, choose that one. For any other bucket, at least 2 SSTables is enough. + * In any case, limit to maxThreshold SSTables. + * + * @param buckets A map from a bucket id to a set of tables, sorted by id and then by table size * @param minThreshold minimum number of sstables in a bucket to qualify. * @param maxThreshold maximum number of sstables to compact at once (the returned bucket will be trimmed down to this). - * @return a bucket (list) of sstables to compact. + * @param stcsOptions the options for {@link SizeTieredCompactionStrategy} to be used in the newest bucket + * @param now the latest timestamp in milliseconds + * + * @return a list of compaction aggregates, one per time bucket */ @VisibleForTesting - static NewestBucket newestBucket(HashMultimap buckets, int minThreshold, int maxThreshold, SizeTieredCompactionStrategyOptions stcsOptions, long now) + static List getBucketAggregates(NavigableMap> buckets, + int minThreshold, + int maxThreshold, + SizeTieredCompactionStrategyOptions stcsOptions, + long now) { - // If the current bucket has at least minThreshold SSTables, choose that one. - // For any other bucket, at least 2 SSTables is enough. - // In any case, limit to maxThreshold SSTables. - - List sstables = Collections.emptyList(); - int estimatedRemainingTasks = 0; + List ret = new ArrayList<>(buckets.size()); + boolean nextCompactionFound = false; // set to true once the first bucket with a compaction is found - TreeSet allKeys = new TreeSet<>(buckets.keySet()); - - Iterator it = allKeys.descendingIterator(); - while(it.hasNext()) + for (Map.Entry> entry : buckets.descendingMap().entrySet()) { - Long key = it.next(); - Set bucket = buckets.get(key); + Long key = entry.getKey(); + List bucket = entry.getValue(); logger.trace("Key {}, now {}", key, now); + + CompactionPick selected = CompactionPick.EMPTY; + List pending = new ArrayList<>(1); + if (bucket.size() >= minThreshold && key >= now) { // If we're in the newest bucket, we'll use STCS to prioritize sstables - List> pairs = SizeTieredCompactionStrategy.createSSTableAndLengthPairs(bucket); - List> stcsBuckets = SizeTieredCompactionStrategy.getBuckets(pairs, stcsOptions.bucketHigh, stcsOptions.bucketLow, stcsOptions.minSSTableSize); - List stcsInterestingBucket = SizeTieredCompactionStrategy.mostInterestingBucket(stcsBuckets, minThreshold, maxThreshold); + SizeTieredCompactionStrategy.SizeTieredBuckets stcsBuckets = new SizeTieredCompactionStrategy.SizeTieredBuckets(bucket, + stcsOptions, + minThreshold, + maxThreshold); + stcsBuckets.aggregate(); - // If the tables in the current bucket aren't eligible in the STCS strategy, we'll skip it and look for other buckets - if (!stcsInterestingBucket.isEmpty()) + for (CompactionAggregate stcsAggregate : stcsBuckets.getAggregates()) { - double remaining = bucket.size() - maxThreshold; - estimatedRemainingTasks += 1 + (remaining > minThreshold ? Math.ceil(remaining / maxThreshold) : 0); - if (sstables.isEmpty()) + if (selected.isEmpty()) { - logger.debug("Using STCS compaction for first window of bucket: data files {} , options {}", pairs, stcsOptions); - sstables = stcsInterestingBucket; + selected = CompactionPick.create(key, stcsAggregate.getSelected()); + for (CompactionPick comp : stcsAggregate.getActive()) + { + if (comp != stcsAggregate.getSelected()) + pending.add(comp); + } } else { - logger.trace("First window of bucket is eligible but not selected: data files {} , options {}", pairs, stcsOptions); + pending.addAll(stcsAggregate.getActive()); } } + + if (!selected.isEmpty()) + logger.debug("Newest window has STCS compaction candidates, {}, data files {} , options {}", + nextCompactionFound ? "eligible but not selected due to prior candidate" : "will be selected for compaction", + stcsBuckets.pairs(), + stcsOptions); + else + logger.debug("No STCS compactions found for first window, data files {}, options {}", stcsBuckets.pairs(), stcsOptions); + + if (!nextCompactionFound && !selected.isEmpty()) + { + nextCompactionFound = true; + ret.add(0, CompactionAggregate.createTimeTiered(bucket, selected, pending, key)); // the first one will be submitted for compaction + } + else + { + ret.add(CompactionAggregate.createTimeTiered(bucket, selected, pending, key)); + } } else if (bucket.size() >= 2 && key < now) { - double remaining = bucket.size() - maxThreshold; - estimatedRemainingTasks += 1 + (remaining > minThreshold ? Math.ceil(remaining / maxThreshold) : 0); - if (sstables.isEmpty()) + List sstables = bucket; + + // Sort the largest sstables off the end before splitting by maxThreshold + Collections.sort(sstables, SSTableReader.sizeComparator); + + int i = 0; + while ((bucket.size() - i) >= 2) + { + List pick = sstables.subList(i, i + Math.min(bucket.size() - i, maxThreshold)); + if (selected.isEmpty()) + selected = CompactionPick.create(key, pick); + else + pending.add(CompactionPick.create(key, pick)); + + i += pick.size(); + } + + if (!nextCompactionFound) { logger.debug("bucket size {} >= 2 and not in current bucket, compacting what's here: {}", bucket.size(), bucket); - sstables = trimToThreshold(bucket, maxThreshold); + nextCompactionFound = true; + ret.add(0, CompactionAggregate.createTimeTiered(bucket, selected, pending, key)); // the first one will be submitted for compaction } else { logger.trace("bucket size {} >= 2 and not in current bucket, eligible but not selected: {}", bucket.size(), bucket); + ret.add(CompactionAggregate.createTimeTiered(bucket, selected, pending, key)); } } else { logger.trace("No compaction necessary for bucket size {} , key {}, now {}", bucket.size(), key, now); + ret.add(CompactionAggregate.createTimeTiered(bucket, selected, pending, key)); // add an empty aggregate anyway so we get a full view } } - return new NewestBucket(sstables, estimatedRemainingTasks); - } - - /** - * @param bucket set of sstables - * @param maxThreshold maximum number of sstables in a single compaction task. - * @return A bucket trimmed to the maxThreshold newest sstables. - */ - @VisibleForTesting - static List trimToThreshold(Set bucket, int maxThreshold) - { - List ssTableReaders = new ArrayList<>(bucket); - - // Trim the largest sstables off the end to meet the maxThreshold - Collections.sort(ssTableReaders, SSTableReader.sizeComparator); - - return ImmutableList.copyOf(Iterables.limit(ssTableReaders, maxThreshold)); - } - - @Override - @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute - public synchronized Collection getMaximalTask(int gcBefore, boolean splitOutput) - { - Iterable filteredSSTables = filterSuspectSSTables(sstables); - if (Iterables.isEmpty(filteredSSTables)) - return null; - LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION); - if (txn == null) - return null; - return Collections.singleton(new TimeWindowCompactionTask(cfs, txn, gcBefore, options.ignoreOverlaps)); - } - - @Override - @SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute - public synchronized AbstractCompactionTask getUserDefinedTask(Collection sstables, int gcBefore) - { - assert !sstables.isEmpty(); // checked for by CM.submitUserDefined - - LifecycleTransaction modifier = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION); - if (modifier == null) - { - logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables); - return null; - } - - return new TimeWindowCompactionTask(cfs, modifier, gcBefore, options.ignoreOverlaps).setUserDefined(true); + return ret; } - public int getEstimatedRemainingTasks() + boolean ignoreOverlaps() { - return this.estimatedRemainingTasks; + return twcsOptions.ignoreOverlaps; } public long getMaxSSTableBytes() @@ -409,7 +383,7 @@ public long getMaxSSTableBytes() public static Map validateOptions(Map options) throws ConfigurationException { - Map uncheckedOptions = AbstractCompactionStrategy.validateOptions(options); + Map uncheckedOptions = CompactionStrategyOptions.validateOptions(options); uncheckedOptions = TimeWindowCompactionStrategyOptions.validateOptions(options, uncheckedOptions); uncheckedOptions.remove(CompactionParams.Option.MIN_THRESHOLD.toString()); diff --git a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionTask.java index 4f1fe6a0b998..a2c10d9088a2 100644 --- a/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionTask.java +++ b/src/java/org/apache/cassandra/db/compaction/TimeWindowCompactionTask.java @@ -28,9 +28,9 @@ public class TimeWindowCompactionTask extends CompactionTask { private final boolean ignoreOverlaps; - public TimeWindowCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean ignoreOverlaps) + public TimeWindowCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean ignoreOverlaps, TimeWindowCompactionStrategy strategy) { - super(cfs, txn, gcBefore); + super(cfs, txn, gcBefore, false, strategy); this.ignoreOverlaps = ignoreOverlaps; } diff --git a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionContainer.java b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionContainer.java new file mode 100644 index 000000000000..def929cb3ae3 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionContainer.java @@ -0,0 +1,361 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.ScannerList; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.notifications.INotification; +import org.apache.cassandra.schema.CompactionParams; + +public class UnifiedCompactionContainer implements CompactionStrategyContainer +{ + private final CompactionStrategyFactory factory; + private final CompactionParams params; + private final CompactionParams metadataParams; + private final UnifiedCompactionStrategy strategy; + + AtomicBoolean enabled; + + UnifiedCompactionContainer(CompactionStrategyFactory factory, + BackgroundCompactions backgroundCompactions, + CompactionParams params, + CompactionParams metadataParams, + boolean enabled) + { + this.factory = factory; + this.params = params; + this.metadataParams = metadataParams; + this.strategy = new UnifiedCompactionStrategy(factory, backgroundCompactions, params.options()); + this.enabled = new AtomicBoolean(enabled); + + factory.getCompactionLogger().strategyCreated(this.strategy); + + if (this.strategy.getOptions().isLogAll()) + factory.getCompactionLogger().enable(); + else + factory.getCompactionLogger().disable(); + + startup(); + } + + @Override + public void enable() + { + this.enabled.set(true); + } + + @Override + public void disable() + { + this.enabled.set(false); + } + + @Override + public boolean isEnabled() + { + return enabled.get() && strategy.isActive; + } + + @Override + public boolean isActive() + { + return strategy.isActive; + } + + public static CompactionStrategyContainer create(@Nullable CompactionStrategyContainer previous, + CompactionStrategyFactory strategyFactory, + CompactionParams compactionParams, + CompactionStrategyContainer.ReloadReason reason) + { + boolean enabled = CompactionStrategyFactory.enableCompactionOnReload(previous, compactionParams, reason); + BackgroundCompactions backgroundCompactions; + // inherit compactions history from previous UCS container + if (previous instanceof UnifiedCompactionContainer) + backgroundCompactions = ((UnifiedCompactionContainer) previous).getBackgroundCompactions(); + + // for other cases start from scratch + // We don't inherit from legacy compactions right now because there are multiple strategies and we'd need + // to merge their BackgroundCompactions to support that. Merging per se is not tricky, but the bigger problem + // is aggregate cleanup. We'd need to unsubscribe from compaction tasks by legacy strategies and subscribe + // by the new UCS to remove inherited ongoing compactions when they complete. + // We might want to revisit this issue later to improve UX. + else + backgroundCompactions = new BackgroundCompactions(strategyFactory.getCfs()); + CompactionParams metadataParams = createMetadataParams(previous, compactionParams, reason); + + if (previous != null) + previous.shutdown(); + + return new UnifiedCompactionContainer(strategyFactory, + backgroundCompactions, + compactionParams, + metadataParams, + enabled); + } + + @Override + public CompactionStrategyContainer reload(@Nonnull CompactionStrategyContainer previous, + CompactionParams compactionParams, + ReloadReason reason) + { + return create(previous, factory, compactionParams, reason); + } + + private static CompactionParams createMetadataParams(@Nullable CompactionStrategyContainer previous, + CompactionParams compactionParams, + ReloadReason reason) + { + CompactionParams metadataParams; + if (reason == CompactionStrategyContainer.ReloadReason.METADATA_CHANGE) + // metadataParams are aligned with compactionParams. We do not access TableParams.compaction to avoid racing with + // concurrent ALTER TABLE metadata change. + metadataParams = compactionParams; + else if (previous != null) + metadataParams = previous.getMetadataCompactionParams(); + else + metadataParams = null; + + return metadataParams; + } + + @Override + public CompactionParams getCompactionParams() + { + return params; + } + + @Override + public CompactionParams getMetadataCompactionParams() + { + return metadataParams; + } + + @Override + public List getStrategies() + { + return ImmutableList.of(strategy); + } + + @Override + public List getStrategies(boolean isRepaired, @Nullable UUID pendingRepair) + { + return getStrategies(); + } + + @Override + public void repairSessionCompleted(UUID sessionID) + { + // We are not tracking SSTables, so nothing to do here. + } + + /** + * UCC does not need to use this method with {@link ColumnFamilyStore#mutateRepaired} + * @return null + */ + @Override + public ReentrantReadWriteLock.WriteLock getWriteLock() + { + return null; + } + + @Override + public CompactionLogger getCompactionLogger() + { + return strategy.compactionLogger; + } + + @Override + public void pause() + { + strategy.pause(); + } + + @Override + public void resume() + { + strategy.resume(); + } + + @Override + public void startup() + { + strategy.startup(); + } + + @Override + public void shutdown() + { + strategy.shutdown(); + } + + @Override + public Collection getNextBackgroundTasks(int gcBefore) + { + return strategy.getNextBackgroundTasks(gcBefore); + } + + @Override + public CompactionTasks getMaximalTasks(int gcBefore, boolean splitOutput) + { + return strategy.getMaximalTasks(gcBefore, splitOutput); + } + + @Override + public CompactionTasks getUserDefinedTasks(Collection sstables, int gcBefore) + { + return strategy.getUserDefinedTasks(sstables, gcBefore); + } + + @Override + public int getEstimatedRemainingTasks() + { + return strategy.getEstimatedRemainingTasks(); + } + + @Override + public AbstractCompactionTask createCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes) + { + return strategy.createCompactionTask(txn, gcBefore, maxSSTableBytes); + } + + @Override + public int getTotalCompactions() + { + return strategy.getTotalCompactions(); + } + + @Override + public List getStatistics() + { + return strategy.getStatistics(); + } + + @Override + public long getMaxSSTableBytes() + { + return strategy.getMaxSSTableBytes(); + } + + @Override + public int[] getSSTableCountPerLevel() + { + return strategy.getSSTableCountPerLevel(); + } + + @Override + public int getLevelFanoutSize() + { + return strategy.getLevelFanoutSize(); + } + + @Override + public ScannerList getScanners(Collection sstables, Collection> ranges) + { + return strategy.getScanners(sstables, ranges); + } + + @Override + public String getName() + { + return strategy.getName(); + } + + @Override + public Set getSSTables() + { + return strategy.getSSTables(); + } + + @Override + public Collection> groupSSTablesForAntiCompaction(Collection sstablesToGroup) + { + return strategy.groupSSTablesForAntiCompaction(sstablesToGroup); + } + + @Override + public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, + long keyCount, + long repairedAt, + UUID pendingRepair, + boolean isTransient, + MetadataCollector collector, + SerializationHeader header, + Collection indexGroups, + LifecycleNewTracker lifecycleNewTracker) + { + return strategy.createSSTableMultiWriter(descriptor, + keyCount, + repairedAt, + pendingRepair, + isTransient, + collector, + header, + indexGroups, + lifecycleNewTracker); + } + + @Override + public boolean supportsEarlyOpen() + { + return strategy.supportsEarlyOpen(); + } + + BackgroundCompactions getBackgroundCompactions() + { + return strategy.backgroundCompactions; + } + + @Override + public void onInProgress(CompactionProgress progress) + { + strategy.onInProgress(progress); + } + + @Override + public void onCompleted(UUID id) + { + strategy.onCompleted(id); + } + + @Override + public void handleNotification(INotification notification, Object sender) + { + // TODO - this is a no-op because the strategy is stateless but we could detect here + // sstables that are added either because of streaming or because of nodetool refresh + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStatistics.java b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStatistics.java new file mode 100644 index 000000000000..e4f2dde3f607 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStatistics.java @@ -0,0 +1,165 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.cassandra.utils.FBUtilities; + +/** + * The statistics for size tiered compaction. + *

    + * Implements serializable to allow structured info to be returned via JMX. + */ +public class UnifiedCompactionStatistics extends CompactionAggregateStatistics +{ + private static final Collection HEADER = ImmutableList.copyOf(Iterables.concat(ImmutableList.of("Bucket", "W", "T", "F", "min size", "max size"), + CompactionAggregateStatistics.HEADER)); + + private static final long serialVersionUID = 3695927592357345266L; + + /** The bucket number */ + private final int bucket; + + /** The survival factor o */ + private final double survivalFactor; + + /** The scaling parameter W */ + private final int scalingParameter; + + /** The number of SSTables T that trigger a compaction */ + private final int threshold; + + /** The fanout size F */ + private final int fanout; + + /** The minimum size for an SSTable that belongs to this bucket */ + private final long minSizeBytes; + + /** The maximum size for an SSTable run that belongs to this bucket */ + private final long maxSizeBytes; + + /** The name of the shard */ + private final String shard; + + UnifiedCompactionStatistics(CompactionAggregateStatistics base, + int bucketIndex, + double survivalFactor, + int scalingParameter, + int threshold, + int fanout, + long minSizeBytes, + long maxSizeBytes, + String shard) + { + super(base); + + this.bucket = bucketIndex; + this.survivalFactor = survivalFactor; + this.scalingParameter = scalingParameter; + this.threshold = threshold; + this.fanout = fanout; + this.minSizeBytes = minSizeBytes; + this.maxSizeBytes = maxSizeBytes; + this.shard = shard; + } + + /** The bucket number */ + @JsonProperty + public int bucket() + { + return bucket; + } + + /** The survival factor o, currently always one */ + @JsonProperty + public double survivalFactor() + { + return survivalFactor; + } + + /** The scaling parameter W */ + @JsonProperty + public int scalingParameter() + { + return scalingParameter; + } + + /** The number of SSTables T that trigger a compaction */ + @JsonProperty + public int threshold() + { + return threshold; + } + + /** The fanout size F */ + @JsonProperty + public int fanout() + { + return fanout; + } + + /** The minimum size for an SSTable that belongs to this bucket */ + @JsonProperty + public long minSizeBytes() + { + return minSizeBytes; + } + + /** The maximum size for an SSTable that belongs to this bucket */ + @JsonProperty + public long maxSizeBytes() + { + return maxSizeBytes; + } + + /** The name of the shard, empty if the compaction is not sharded (the default). */ + @JsonProperty + @Override + public String shard() + { + return shard; + } + + @Override + protected Collection header() + { + return HEADER; + } + + @Override + protected Collection data() + { + List data = new ArrayList<>(HEADER.size()); + data.add(Integer.toString(bucket())); + data.add(Integer.toString(scalingParameter)); + data.add(Integer.toString(threshold)); + data.add(Integer.toString(fanout)); + data.add(FBUtilities.prettyPrintMemory(minSizeBytes)); + data.add(FBUtilities.prettyPrintMemory(maxSizeBytes)); + + data.addAll(super.data()); + + return data; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java new file mode 100644 index 000000000000..d090b27dc474 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/UnifiedCompactionStrategy.java @@ -0,0 +1,1076 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Ordering; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.agrona.collections.IntArrayList; +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DiskBoundaries; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.SortedLocalRanges; +import org.apache.cassandra.db.compaction.unified.Controller; +import org.apache.cassandra.db.compaction.unified.ShardedMultiWriter; +import org.apache.cassandra.db.compaction.unified.UnifiedCompactionTask; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Splitter; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.utils.Throwables.perform; + +/** + * The unified compaction strategy is described in this design document: + * + * TODO: link to design doc or SEP + */ +public class UnifiedCompactionStrategy extends AbstractCompactionStrategy +{ + public static final Class CONTAINER_CLASS = UnifiedCompactionContainer.class; + + private static final Logger logger = LoggerFactory.getLogger(UnifiedCompactionStrategy.class); + + static final int MAX_LEVELS = 32; // This is enough for a few petabytes of data (with the worst case fan factor + // at W=0 this leaves room for 2^32 sstables, presumably of at least 1MB each). + + private final Controller controller; + + private volatile ArenaSelector arenaSelector; + + private long lastExpiredCheck; + + public UnifiedCompactionStrategy(CompactionStrategyFactory factory, BackgroundCompactions backgroundCompactions, Map options) + { + this(factory, backgroundCompactions, options, Controller.fromOptions(factory.getCfs(), options)); + } + + public UnifiedCompactionStrategy(CompactionStrategyFactory factory, BackgroundCompactions backgroundCompactions, Controller controller) + { + this(factory, backgroundCompactions, new HashMap<>(), controller); + } + + public UnifiedCompactionStrategy(CompactionStrategyFactory factory, BackgroundCompactions backgroundCompactions, Map options, Controller controller) + { + super(factory, backgroundCompactions, options); + this.controller = controller; + } + + @VisibleForTesting + public UnifiedCompactionStrategy(CompactionStrategyFactory factory, Controller controller) + { + this(factory, new BackgroundCompactions(factory.getCfs()), new HashMap<>(), controller); + } + + public static Map validateOptions(Map options) throws ConfigurationException + { + return Controller.validateOptions(CompactionStrategyOptions.validateOptions(options)); + } + + @Override + public Collection> groupSSTablesForAntiCompaction(Collection sstablesToGroup) + { + Collection> groups = new ArrayList<>(); + for (Shard shard : getCompactionShards(sstablesToGroup)) + { + groups.addAll(super.groupSSTablesForAntiCompaction(shard.sstables)); + } + + return groups; + } + + @Override + public void startup() + { + perform(super::startup, + () -> controller.startup(this, ScheduledExecutors.scheduledTasks)); + } + + @Override + public void shutdown() + { + perform(super::shutdown, + controller::shutdown); + } + + /** + * Returns a collections of compaction tasks. + * + * This method is synchornized because task creation is significantly more expensive in UCS; the strategy is + * stateless, therefore it has to compute the shard/bucket structure on each call. + * + * @param gcBefore throw away tombstones older than this + * @return collection of AbstractCompactionTask, which could be either a CompactionTask or an UnifiedCompactionTask + */ + @Override + public synchronized Collection getNextBackgroundTasks(int gcBefore) + { + controller.onStrategyBackgroundTaskRequest(); + + Collection compactionAggregates = getNextCompactionAggregates(gcBefore); + + Collection tasks = new ArrayList<>(compactionAggregates.size()); + for (CompactionAggregate aggregate : compactionAggregates) + { + LifecycleTransaction transaction = dataTracker.tryModify(aggregate.getSelected().sstables, OperationType.COMPACTION); + if (transaction != null) + { + backgroundCompactions.setSubmitted(this, transaction.opId(), aggregate); + tasks.add(createCompactionTask(transaction, gcBefore)); + } + else + { + // This can happen e.g. due to a race with upgrade tasks + logger.error("Failed to submit compaction {} because a transaction could not be created. If this happens frequently, it should be reported", aggregate); + } + } + + return tasks; + } + + /** + * Create the sstable writer used for flushing. + * + * @return either a normal sstable writer, if there are no shards, or a sharded sstable writer that will + * create multiple sstables if a shard has a sufficiently large sstable. + */ + @Override + public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, + long keyCount, + long repairedAt, + UUID pendingRepair, + boolean isTransient, + MetadataCollector meta, + SerializationHeader header, + Collection indexGroups, + LifecycleNewTracker lifecycleNewTracker) + { + if (controller.getNumShards() <= 1) + return super.createSSTableMultiWriter(descriptor, + keyCount, + repairedAt, + pendingRepair, + isTransient, + meta, + header, + indexGroups, + lifecycleNewTracker); + + return new ShardedMultiWriter(cfs, + descriptor, + keyCount, + repairedAt, + pendingRepair, + isTransient, + meta, + header, + indexGroups, + lifecycleNewTracker, + controller.getMinSstableSizeBytes(), + getShardBoundaries()); + } + + /** + * Create the task that in turns creates the sstable writer used for compaction. + * + * @return either a normal compaction task, if there are no shards, or a sharded compaction task that in turn will + * create a sharded compaction writer. + */ + private CompactionTask createCompactionTask(LifecycleTransaction transaction, int gcBefore) + { + if (controller.getNumShards() <= 1) + return new CompactionTask(cfs, transaction, gcBefore, false, this); + + return new UnifiedCompactionTask(cfs, this, transaction, gcBefore, controller.getMinSstableSizeBytes(), getShardBoundaries()); + } + + private void maybeUpdateSelector() + { + if (arenaSelector != null && !arenaSelector.diskBoundaries.isOutOfDate()) + return; // the disk boundaries (and thus the local ranges too) have not changed since the last time we calculated + + synchronized (this) + { + if (arenaSelector != null && !arenaSelector.diskBoundaries.isOutOfDate()) + return; // another thread beat us to the update + + DiskBoundaries currentBoundaries = cfs.getDiskBoundaries(); + List shardBoundaries = computeShardBoundaries(currentBoundaries.getLocalRanges(), + currentBoundaries.getPositions(), + controller.getNumShards(), + cfs.getPartitioner()); + arenaSelector = new ArenaSelector(currentBoundaries, shardBoundaries); + // Note: this can just as well be done without the synchronization (races would be benign, just doing some + // redundant work). For the current usages of this blocking is fine and expected to perform no worse. + } + } + + /** + * We want to split the local token range in shards, aiming for close to equal share for each shard. + * If there are no disk boundaries, we just split the token space equally, but if multiple disks have been defined + * (each with its own share of the local range), we can't have shards spanning disk boundaries. This means that + * shards need to be selected within the disk's portion of the local ranges. + * + * As an example of what this means, consider a 3-disk node and 10 shards. The range is split equally between + * disks, but we can only split shards within a disk range, thus we end up with 6 shards taking 1/3*1/3=1/9 of the + * token range, and 4 smaller shards taking 1/3*1/4=1/12 of the token range. + */ + @VisibleForTesting + static List computeShardBoundaries(SortedLocalRanges localRanges, + List diskBoundaries, + int numShards, + IPartitioner partitioner) + { + Optional splitter = partitioner.splitter(); + if (diskBoundaries != null && !splitter.isPresent()) + return diskBoundaries; + else if (!splitter.isPresent()) // C* 2i case, just return 1 boundary at min token + return ImmutableList.of(partitioner.getMinimumToken().minKeyBound()); + + // this should only happen in tests that change partitioners, but we don't want UCS to throw + // where other strategies work even if the situations are unrealistic. + if (localRanges.getRanges().isEmpty() || !localRanges.getRanges() + .get(0) + .range() + .left + .getPartitioner() + .equals(partitioner)) + localRanges = new SortedLocalRanges(StorageService.instance, + localRanges.getCfs(), + localRanges.getRingVersion(), + ImmutableList.of(new Splitter.WeightedRange(1.0, + new Range<>(partitioner.getMinimumToken(), + partitioner.getMaximumToken())))); + + if (diskBoundaries == null || diskBoundaries.size() <= 1) + return localRanges.split(numShards); + + if (numShards <= diskBoundaries.size()) + return diskBoundaries; + + return splitPerDiskRanges(localRanges, + diskBoundaries, + getRangesTotalSize(localRanges.getRanges()), + numShards, + splitter.get()); + } + + /** + * Split the per-disk ranges and generate the required number of shard boundaries. + * This works by accumulating the size after each disk's share, multiplying by shardNum/totalSize and rounding to + * produce an integer number of total shards needed by the disk boundary, which in turns defines how many need to be + * added for this disk. + * + * For example, for a total size of 1, 2 disks (each of 0.5 share) and 3 shards, this will: + * -process disk 1: + * -- calculate 1/2 as the accumulated size + * -- map this to 3/2 and round to 2 shards + * -- split the disk's ranges into two equally-sized shards + * -process disk 2: + * -- calculate 1 as the accumulated size + * -- map it to 3 and round to 3 shards + * -- assign the disk's ranges to one shard + * + * The resulting shards will not be of equal size and this works best if the disk shares are distributed evenly + * (which the current code always ensures). + */ + private static List splitPerDiskRanges(SortedLocalRanges localRanges, + List diskBoundaries, + double totalSize, + int numShards, + Splitter splitter) + { + double perShard = totalSize / numShards; + List shardBoundaries = new ArrayList<>(numShards); + double processedSize = 0; + Token left = diskBoundaries.get(0).getToken().getPartitioner().getMinimumToken(); + for (PartitionPosition boundary : diskBoundaries) + { + Token right = boundary.getToken(); + List disk = localRanges.subrange(new Range<>(left, right)); + + processedSize += getRangesTotalSize(disk); + int targetCount = (int) Math.round(processedSize / perShard); + List splits = splitter.splitOwnedRanges(Math.max(targetCount - shardBoundaries.size(), 1), disk, Splitter.SplitType.ALWAYS_SPLIT).boundaries; + shardBoundaries.addAll(Collections2.transform(splits, Token::maxKeyBound)); + // The splitting always results in maxToken as the last boundary. Replace it with the disk's upper bound. + shardBoundaries.set(shardBoundaries.size() - 1, boundary); + + left = right; + } + assert shardBoundaries.size() == numShards; + return shardBoundaries; + } + + private static double getRangesTotalSize(List ranges) + { + double totalSize = 0; + for (Splitter.WeightedRange range : ranges) + totalSize += range.left().size(range.right()); + return totalSize; + } + + @VisibleForTesting + List getShardBoundaries() + { + maybeUpdateSelector(); + return arenaSelector.shardBoundaries; + } + + private Collection getNextCompactionAggregates(int gcBefore) + { + // Calculate the running compaction limits, i.e. the overall number of compactions permitted, which is either + // the compaction thread count, or the compaction throughput divided by the compaction rate (to prevent slowing + // down individual compaction progress). + String rateLimitLog = ""; + + // identify parallel compactions limit + int maxConcurrentCompactions = controller.maxConcurrentCompactions(); + long spaceOverheadLimit = controller.maxCompactionSpaceBytes(); + + // identify throughput limit + double throughputLimit = controller.maxThroughput(); + int maxCompactions; + if (throughputLimit < Double.MAX_VALUE) + { + int maxCompactionsForThroughput; + + double compactionRate = backgroundCompactions.compactionRate.get(); + if (compactionRate > 0) + { + // Start as many as can saturate the limit, making sure to also account for compactions that have + // already been started but don't have progress yet. + + // Note: the throughput limit is adjusted here because the limiter won't let compaction proceed at more + // than the given rate, and small hiccups or rounding errors could cause this to go above the current + // running count when we are already at capacity. + // Allow up to 5% variability, or if we are permitted more than 20 concurrent compactions, one/maxcount + // so that we don't issue less tasks than we should. + double adjustment = Math.min(0.05, 1.0 / maxConcurrentCompactions); + maxCompactionsForThroughput = (int) Math.ceil(throughputLimit * (1 - adjustment) / compactionRate); + } + else + { + // If we don't have running compactions we don't know the effective rate. + // Allow only one compaction; this will be called again soon enough to recheck. + maxCompactionsForThroughput = 1; + } + + rateLimitLog = String.format(" rate-based limit %d (rate %s/%s)", + maxCompactionsForThroughput, + FBUtilities.prettyPrintMemoryPerSecond((long) compactionRate), + FBUtilities.prettyPrintMemoryPerSecond((long) throughputLimit)); + maxCompactions = Math.min(maxConcurrentCompactions, maxCompactionsForThroughput); + } + else + maxCompactions = maxConcurrentCompactions; + + // Now that we have a count, make sure it is spread close to equally among levels. In other words, reserve + // floor(permitted / levels) compactions for each level and don't permit more than ceil(permitted / levels) on + // any, to make sure that no level hogs all threads and thus lowest-level ops (which need to run more often but + // complete quickest) have a chance to run frequently. Also, running compactions can't go above the specified + // space overhead limit. + // To do this we count the number and size of already running compactions on each level and make sure any new + // ones we select satisfy these constraints. + int[] perLevel = new int[MAX_LEVELS]; + int levelCount = 1; // Start at 1 to avoid division by zero if the aggregates list is empty. + int runningCompactions = 0; + long spaceAvailable = spaceOverheadLimit; + for (CompactionPick compaction : backgroundCompactions.getCompactionsInProgress()) + { + final int level = levelOf(compaction); + ++perLevel[level]; + ++runningCompactions; + levelCount = Math.max(levelCount, level + 1); + spaceAvailable -= compaction.totSizeInBytes; + } + + logger.debug("Selecting up to {} new compactions of up to {}, concurrency limit {}{}", + maxCompactions - runningCompactions, + FBUtilities.prettyPrintMemory(spaceAvailable), + maxConcurrentCompactions, + rateLimitLog); + + List pending = new ArrayList<>(); + long ts = System.currentTimeMillis(); + boolean expiredCheck = ts - lastExpiredCheck > controller.getExpiredSSTableCheckFrequency(); + if (expiredCheck) + lastExpiredCheck = ts; + + for (Map.Entry> entry : getShardsWithBuckets().entrySet()) + { + Shard shard = entry.getKey(); + Set expired; + if (expiredCheck) + { + expired = shard.getExpiredSSTables(gcBefore, controller.getIgnoreOverlapsInExpirationCheck()); + if (logger.isTraceEnabled() && expired.size() > 0) + logger.trace("Expiration check for shard {} found {} fully expired SSTables", shard.name(), expired.size()); + } + else + expired = Collections.emptySet(); + + for (Bucket bucket : entry.getValue()) + { + CompactionAggregate.UnifiedAggregate aggregate = bucket.getCompactionAggregate(shard, expired, controller, spaceAvailable); + // Note: We allow empty aggregates into the list of pending compactions. The pending compactions list + // is for progress tracking only, and it is helpful to see empty levels there. + pending.add(aggregate); + + // Make sure the level count includes all levels for which we have sstables (to be ready to compact + // as soon as the threshold is crossed)... + levelCount = Math.max(levelCount, aggregate.bucketIndex() + 1); + if (aggregate.selected != null) + { + // ... and also the levels that a layout-preserving selection would create. + levelCount = Math.max(levelCount, levelOf(aggregate.selected) + 1); + } + + // The space overhead limit also applies when a single compaction is above that limit. This should + // prevent running out of space at the expense of several highest-level tables extra, i.e. slightly + // higher read amplification, which I think is a sensible tradeoff; however, operators must be warned + // if this happens. + warnIfSizeAbove(aggregate, spaceOverheadLimit); + } + } + + // Update the tracked background tasks. + backgroundCompactions.setPending(this, pending); + + final List selection = getSelection(pending, maxCompactions, levelCount, perLevel, spaceAvailable); + logger.debug("Starting {} compactions.", selection.size()); + return selection; + } + + private void warnIfSizeAbove(CompactionAggregate.UnifiedAggregate aggregate, long spaceOverheadLimit) + { + if (aggregate.selected.totSizeInBytes > spaceOverheadLimit) + logger.warn("Compaction needs to perform an operation that is bigger than the current space overhead " + + "limit - size {} (compacting {} sstables in shard {}/bucket {}); limit {} = {}% of dataset size {}. " + + "To honor the limit, this operation will not be performed, which may result in degraded performance.\n" + + "Please verify the compaction parameters, specifically {} and {}.", + FBUtilities.prettyPrintMemory(aggregate.selected.totSizeInBytes), + aggregate.selected.sstables.size(), + aggregate.getShard().name(), + aggregate.bucketIndex(), + FBUtilities.prettyPrintMemory(spaceOverheadLimit), + controller.getMaxSpaceOverhead() * 100, + FBUtilities.prettyPrintMemory(controller.getDataSetSizeBytes()), + Controller.DATASET_SIZE_OPTION_GB, + Controller.MAX_SPACE_OVERHEAD_OPTION); + } + + /** + * Returns a random selection of the compactions to be submitted. The selection will be chosen so that the total + * number of compactions is at most totalCount, where each level gets a share that is the whole part of the ratio + * between the the total permitted number of compactions, and the remainder gets distributed randomly among the + * levels. Note that if a level does not have tasks to fill its share, its quota will remain unused in this + * allocation. + * + * The selection also limits the size of the newly scheduled compactions to be below spaceAvailable by not + * scheduling compactions if they would push the combined size above that limit. + * + * @param pending list of all current aggregates with possible selection for each bucket + * @param totalCount maximum number of compactions permitted to run + * @param levelCount number of levels in use + * @param perLevel int array with the number of in-progress compactions per level + * @param spaceAvailable amount of space in bytes available for the new compactions + */ + List getSelection(List pending, + int totalCount, + int levelCount, + int[] perLevel, + long spaceAvailable) + { + int perLevelCount = totalCount / levelCount; // each level has this number of tasks reserved for it + int remainder = totalCount % levelCount; // and the remainder is distributed randomly, up to 1 per level + + // List the indexes of all compaction picks, adding several entries for compactions that span multiple shards. + IntArrayList list = new IntArrayList(pending.size(), -1); + IntArrayList expired = new IntArrayList(pending.size(), -1); + for (int aggregateIndex = 0; aggregateIndex < pending.size(); ++aggregateIndex) + { + CompactionAggregate.UnifiedAggregate aggregate = pending.get(aggregateIndex); + final CompactionPick pick = aggregate.selected; + if (pick.isEmpty()) + continue; + if (pick.hasExpiredOnly()) + { + expired.add(aggregateIndex); + continue; + } + if (pick.totSizeInBytes > spaceAvailable) + continue; + if (perLevel[levelOf(pick)] > perLevelCount) + continue; // this level is already using up all its share + one, we can ignore candidate altogether + + int shardsSpanned = shardsSpanned(pick); + for (int i = 0; i < shardsSpanned; ++i) // put an entry for each spanned shard + list.addInt(aggregateIndex); + } + if (list.isEmpty() && expired.isEmpty()) + return ImmutableList.of(); + + BitSet selection = new BitSet(pending.size()); + + // Always include expire-only aggregates + for (int i = 0; i < expired.size(); i++) + selection.set(expired.get(i)); + + int selectedSize = 0; + if (!list.isEmpty()) + { + // Randomize the list. + Collections.shuffle(list, controller.random()); + + // Calculate how many new ones we can add in each level, and how many we can assign randomly. + int remaining = totalCount; + for (int i = 0; i < levelCount; ++i) + { + remaining -= perLevel[i]; + if (perLevel[i] > perLevelCount) + remainder -= perLevel[i] - perLevelCount; + } + int toAdd = remaining; + // Note: if we are in the middle of changes in the parameters or level count, remainder might become negative. + // This is okay, some buckets will temporarily not get their rightful share until these tasks complete. + + // Select the first ones, skipping over duplicates and permitting only the specified number per level. + for (int i = 0; remaining > 0 && i < list.size(); ++i) + { + final int aggregateIndex = list.getInt(i); + if (selection.get(aggregateIndex)) + continue; // this is a repeat + CompactionAggregate.UnifiedAggregate aggregate = pending.get(aggregateIndex); + if (aggregate.selected.totSizeInBytes > spaceAvailable) + continue; // compaction is too large for current cycle + int level = levelOf(aggregate.selected); + + if (perLevel[level] > perLevelCount) + continue; // share + one already used + else if (perLevel[level] == perLevelCount) + { + if (remainder <= 0) + continue; // share used up, no remainder to distribute + --remainder; + } + + --remaining; + ++perLevel[level]; + spaceAvailable -= aggregate.selected.totSizeInBytes; + selection.set(aggregateIndex); + } + + selectedSize = toAdd - remaining; + } + + // Return in the order of the pending aggregates to satisfy tests. + List aggregates = new ArrayList<>(selectedSize + expired.size()); + for (int i = selection.nextSetBit(0); i >= 0; i = selection.nextSetBit(i+1)) + aggregates.add(pending.get(i)); + + return aggregates; + } + + private int shardsSpanned(CompactionPick pick) + { + DecoratedKey min = pick.sstables.stream().map(SSTableReader::getFirst).min(Ordering.natural()).get(); + DecoratedKey max = pick.sstables.stream().map(SSTableReader::getLast).max(Ordering.natural()).get(); + return arenaSelector.shardFor(max) - arenaSelector.shardFor(min) + 1; + } + + @Override + public int getEstimatedRemainingTasks() + { + return backgroundCompactions.getEstimatedRemainingTasks(); + } + + @Override + public long getMaxSSTableBytes() + { + return Long.MAX_VALUE; + } + + @Override + public Set getSSTables() + { + return dataTracker.getLiveSSTables(); + } + + @VisibleForTesting + public int getW(int index) + { + return controller.getScalingParameter(index); + } + + @VisibleForTesting + public Controller getController() + { + return controller; + } + + /** + * Group candidate sstables (non suspect and not already compacting, and not an early version of a compaction + * result) into one or more compaction shards. Each compaction shard is obtained by comparing using a compound + * comparator for the equivalence classes. + * + * @return a list of shards, where each shard contains sstables that are eligible for being compacted together + */ + @VisibleForTesting + Collection getCompactionShards() + { + return getCompactionShards(dataTracker.getLiveSSTables()); + } + + Collection getCompactionShards(Collection sstables) + { + final ArenaSelector arenaSelector = this.arenaSelector; + Map tables = new TreeMap<>(arenaSelector); + for (SSTableReader table : sstables) + if (isSuitableForCompaction(table)) + tables.computeIfAbsent(table, t -> new Shard(arenaSelector, cfs)) + .add(table); + + return tables.values(); + } + + private boolean isSuitableForCompaction(SSTableReader r) + { + return !r.isMarkedSuspect() + && r.openReason != SSTableReader.OpenReason.EARLY + && !dataTracker.getCompacting().contains(r); + } + + /** + * @return a LinkedHashMap of shards with buckets where order of shards are preserved + */ + @VisibleForTesting + Map> getShardsWithBuckets() + { + maybeUpdateSelector(); + Collection shards = getCompactionShards(); + Map> ret = new LinkedHashMap<>(); // should preserve the order of shards + + for (Shard shard : shards) + { + List buckets = new ArrayList<>(MAX_LEVELS); + shard.sstables.sort(arenaSelector::compareByShardAdjustedSize); + + int index = 0; + Bucket bucket = new Bucket(controller, index, 0); + for (SSTableReader candidate : shard.sstables) + { + final long size = arenaSelector.shardAdjustedSize(candidate); + if (size < bucket.max) + { + bucket.add(candidate); + continue; + } + + bucket.sort(); + buckets.add(bucket); // add even if empty + + while (true) + { + bucket = new Bucket(controller, ++index, bucket.max); + if (size < bucket.max) + { + bucket.add(candidate); + break; + } + else + { + buckets.add(bucket); // add the empty bucket + } + } + } + + if (!bucket.sstables.isEmpty()) + { + bucket.sort(); + buckets.add(bucket); + } + + if (!buckets.isEmpty()) + ret.put(shard, buckets); + + if (logger.isTraceEnabled()) + logger.trace("Shard {} has {} buckets", shard, buckets.size()); + } + + logger.debug("Found {} shards with buckets for {}.{}", ret.size(), cfs.getKeyspaceName(), cfs.getTableName()); + return ret; + } + + private static int levelOf(CompactionPick pick) + { + return (int) pick.parent; + } + + public TableMetadata getMetadata() + { + return cfs.metadata(); + } + + /** + * A compaction shard contains the list of sstables that belong to this shard as well as the arena + * selector used for comparison. + */ + final static class Shard implements Comparable + { + final List sstables; + final ArenaSelector selector; + private final ColumnFamilyStore cfs; + + Shard(ArenaSelector selector, ColumnFamilyStore cfs) + { + this.cfs = cfs; + this.sstables = new ArrayList<>(); + this.selector = selector; + } + + void add(SSTableReader ssTableReader) + { + sstables.add(ssTableReader); + } + + public String name() + { + SSTableReader t = sstables.get(0); + return selector.name(t); + } + + @Override + public int compareTo(Shard o) + { + return selector.compare(this.sstables.get(0), o.sstables.get(0)); + } + + @Override + public String toString() + { + return String.format("%s, %d sstables", name(), sstables.size()); + } + + /** + * Find fully expired SSTables. Those will be included in the aggregate no matter what. + * @param gcBefore + * @param ignoreOverlaps + * @return expired SSTables + */ + Set getExpiredSSTables(int gcBefore, boolean ignoreOverlaps) + { + return CompactionController.getFullyExpiredSSTables(cfs, + sstables, + cfs.getOverlappingLiveSSTables(sstables), + gcBefore, + ignoreOverlaps); + } + } + + @Override + public String toString() + { + return String.format("Unified strategy %s", getMetadata()); + } + + /** + * A bucket: index, sstables and some properties. + */ + static class Bucket + { + final List sstables; + final int index; + final double survivalFactor; + final int scalingParameter; // scaling parameter used to calculate fanout and threshold + final int fanout; // fanout factor between buckets + final int threshold; // number of SSTables that trigger a compaction + final long min; // min size of sstables for this bucket + final long max; // max size of sstables for this bucket + double avg = 0; // avg size of sstables in this bucket + + Bucket(Controller controller, int index, long minSize) + { + this.index = index; + this.survivalFactor = controller.getSurvivalFactor(); + this.scalingParameter = controller.getScalingParameter(index); + this.fanout = controller.getFanout(index); + this.threshold = controller.getThreshold(index); + this.sstables = new ArrayList<>(threshold); + this.min = minSize; + + double baseSize = minSize; + if (minSize == 0) + baseSize = controller.getBaseSstableSize(fanout); + + this.max = (long) Math.floor(baseSize * fanout * controller.getSurvivalFactor()); + } + + void add(SSTableReader sstable) + { + this.sstables.add(sstable); + this.avg += (sstable.onDiskLength() - avg) / sstables.size(); + } + + void sort() + { + // Always sort by timestamp, older sstables first. If only a subset of the tables is compacted, let it + // be from a contiguous time span to aid whole-sstable expiration. + sstables.sort(Comparator.comparing(SSTableReader::getMaxTimestamp)); + + if (logger.isTraceEnabled()) + logger.trace("Bucket: {}", this); + } + + /** + * Return the compaction aggregate + */ + CompactionAggregate.UnifiedAggregate getCompactionAggregate(Shard shard, + Set allExpiredSSTables, + Controller controller, + long spaceAvailable) + { + List expiredSet = Collections.emptyList(); + List liveSet = sstables; + if (!allExpiredSSTables.isEmpty()) + { + liveSet = new ArrayList<>(); + expiredSet = new ArrayList<>(); + bipartitionSSTables(sstables, allExpiredSSTables, liveSet, expiredSet); + } + + List pending = ImmutableList.of(); + CompactionPick selected; + int count = liveSet.size(); + int maxSSTablesToCompact = Math.max(fanout, controller.maxSSTablesToCompact()); + + if (count < threshold) + { + // We do not have enough sstables for a compaction. + selected = CompactionPick.EMPTY; + } + else if (count <= fanout) + { + /** + * Happy path. We are not late or (for levelled) we are only so late that a compaction now will + * have the same effect as doing levelled compactions one by one. Compact all. We do not cap + * this pick at maxSSTablesToCompact due to an assumption that maxSSTablesToCompact is much + * greater than F. See {@link Controller#MAX_SSTABLES_TO_COMPACT_OPTION} for more details. + */ + selected = CompactionPick.create(index, liveSet); + } + else if (count <= fanout * controller.getFanout(index + 1)) + { + // Compaction is a bit late, but not enough to jump levels via layout compactions. We need a special + // case to cap compaction pick at maxSSTablesToCompact. + selected = CompactionPick.create(index, liveSet.subList(0, Math.min(maxSSTablesToCompact, count))); + if (count - maxSSTablesToCompact >= threshold) + { + pending = new ArrayList<>(); + int start = maxSSTablesToCompact; + int end = Math.min(2 * maxSSTablesToCompact, count); + while (end - start > threshold) + { + pending.add(CompactionPick.create(index, liveSet.subList(start, end))); + start = end; + end = Math.min(end + maxSSTablesToCompact, count); + } + } + } + // We may, however, have accumulated a lot more than T if compaction is very late, or a set of small + // tables was dumped on us (e.g. when converting from legacy LCS or for tests). + else + { + // We need to pick the compactions in such a way that the result of doing them all spreads the data in + // a similar way to how compaction would lay them if it was able to keep up. This means: + // - for tiered compaction (W >= 0), compact in sets of as many as required to get to a level. + // for example, for W=2 and 55 sstables, do 3 compactions of 16 sstables, 1 of 4, and leave the other 3 alone + // - for levelled compaction (W < 0), compact all that would reach a level. + // for W=-2 and 55, this means one compaction of 48, one of 4, and one of 3 sstables. + pending = layoutCompactions(controller, liveSet, (int) Math.min(spaceAvailable / avg, maxSSTablesToCompact)); + // Out of the set of necessary compactions, choose the one to run randomly. This gives a better + // distribution among levels and should result in more compactions running in parallel in a big data + // dump. + assert !pending.isEmpty(); // we only enter this if count > F: layoutCompactions must have selected something to run + int index = controller.random().nextInt(pending.size()); + selected = pending.remove(index); + } + + boolean hasExpiredSSTables = !expiredSet.isEmpty(); + if (hasExpiredSSTables && selected.equals(CompactionPick.EMPTY)) + // overrides default CompactionPick.EMPTY with parent equal to -1 + selected = CompactionPick.create(index, expiredSet, expiredSet); + else if (hasExpiredSSTables) + selected = selected.withExpiredSSTables(expiredSet); + + return CompactionAggregate.createUnified(sstables, selected, pending, shard, this); + } + + /** + * Bipartitions SSTables into liveSet and expiredSet, depending on whether they are present in allExpiredSSTables. + * + * @param sstables list of SSTables in a bucket + * @param allExpiredSSTables set of expired SSTables for all shards/buckets + * @param liveSet empty list that is going to be filled up with SSTables that are not present in {@param allExpiredSSTables} + * @param expiredSet empty list that is going to be filled up with SSTables that are present in {@param allExpiredSSTables} + */ + private static void bipartitionSSTables(List sstables, + Set allExpiredSSTables, + List liveSet, + List expiredSet) + { + for (SSTableReader sstable : sstables) + { + if (allExpiredSSTables.contains(sstable)) + expiredSet.add(sstable); + else + liveSet.add(sstable); + } + } + + private List layoutCompactions(Controller controller, List liveSet, int maxSSTablesToCompact) + { + List pending = new ArrayList<>(); + int pos = layoutCompactions(controller, liveSet, index + 1, fanout, maxSSTablesToCompact, pending); + int size = liveSet.size(); + if (size - pos >= threshold) // can only happen in the levelled case. + { + assert size - pos < maxSSTablesToCompact; // otherwise it should have already been picked + pending.add(CompactionPick.create(index, liveSet.subList(pos, size))); + } + return pending; + } + + /** + * Collects in {@param list} compactions of {@param sstables} such that they land in {@param level} and higher. + * + * Recursively combines SSTables into {@link CompactionPick}s in way that up to {@param maxSSTablesToCompact} + * SSTables are combined to reach the highest possible level, then the rest is combined for the level before, + * etc up to {@param level}. + * + * To agree with what compaction normally does, the first sstables from the list are placed in the picks that + * combine to reach the highest levels. + * + * @param controller + * @param sstables SSTables to compact, sorted by age from old to new + * @param level minimum target level for compactions to land + * @param step - number of source SSTables required to reach level + * @param maxSSTablesToCompact limit on the number of sstables per compaction + * @param list - result list of layout-preserving compaction picks + * @return index of the last used SSTable from {@param sstables}; the number of remaining sstables will be lower + * than step + */ + private int layoutCompactions(Controller controller, + List sstables, + int level, + int step, + int maxSSTablesToCompact, + List list) + { + if (step > sstables.size() || step > maxSSTablesToCompact) + return 0; + + int W = controller.getScalingParameter(level); + int F = controller.getFanout(level); + int pos = layoutCompactions(controller, + sstables, + level + 1, + step * F, + maxSSTablesToCompact, + list); + + int total = sstables.size(); + // step defines the number of source sstables that are needed to reach this level (ignoring overwrites + // and deletions). + // For tiered compaction we will select batches of this many. + int pickSize = step; + if (W < 0) + { + // For levelled compaction all the sstables that would reach this level need to be compacted to one, + // so select the highest multiple of step that is available, but make sure we don't do a compaction + // bigger than the limit. + pickSize *= Math.min(total - pos, maxSSTablesToCompact) / pickSize; + + if (pickSize == 0) // Not enough sstables to reach this level, we can skip the processing below. + return pos; // Note: this cannot happen on the top level, but can on lower ones. + } + + while (pos + pickSize <= total) + { + // Note that we assign these compactions to the level that would normally produce them, which means that + // they won't be taking up threads dedicated to the busy level. + // Normally sstables end up on a level when a compaction on the previous brings their size to the + // threshold (which corresponds to pickSize == step, always the case for tiered); in the case of + // levelled compaction, when we compact more than 1 but less than F sstables on a level (which + // corresponds to pickSize > step), it is an operation that is triggered on the same level. + list.add(CompactionPick.create(pickSize > step ? level : level - 1, + sstables.subList(pos, pos + pickSize))); + pos += pickSize; + } + + // In the levelled case, if we had to adjust pickSize due to maxSSTablesToCompact, there may + // still be enough sstables to reach this level (e.g. if max was enough for 2*step, but we had 3*step). + if (pos + step <= total) + { + pickSize = ((total - pos) / step) * step; + list.add(CompactionPick.create(pickSize > step ? level : level - 1, + sstables.subList(pos, pos + pickSize))); + pos += pickSize; + } + return pos; + } + + @Override + public String toString() + { + return String.format("W: %d, T: %d, F: %d, index: %d, min: %s, max %s, %d sstables", + scalingParameter, threshold, fanout, index, FBUtilities.prettyPrintMemory(min), FBUtilities.prettyPrintMemory(max), sstables.size()); + } + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java index e1406aa7ed97..e35a8d3b886c 100644 --- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java +++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java @@ -20,7 +20,6 @@ import java.io.File; import java.util.*; import java.util.function.LongPredicate; -import java.util.function.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.Sets; @@ -46,7 +45,7 @@ public class Upgrader private final File directory; private final CompactionController controller; - private final CompactionStrategyManager strategyManager; + private final CompactionStrategyContainer strategyContainer; private final long estimatedRows; private final OutputHandler outputHandler; @@ -62,9 +61,9 @@ public Upgrader(ColumnFamilyStore cfs, LifecycleTransaction txn, OutputHandler o this.controller = new UpgradeController(cfs); - this.strategyManager = cfs.getCompactionStrategyManager(); + this.strategyContainer = cfs.getCompactionStrategyContainer(); long estimatedTotalKeys = Math.max(cfs.metadata().params.minIndexInterval, SSTableReader.getApproximateKeyCount(Arrays.asList(this.sstable))); - long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(Arrays.asList(this.sstable)) / strategyManager.getMaxSSTableBytes()); + long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(Arrays.asList(this.sstable)) / strategyContainer.getMaxSSTableBytes()); this.estimatedRows = (long) Math.ceil((double) estimatedTotalKeys / estimatedSSTables); } @@ -80,7 +79,7 @@ private SSTableWriter createCompactionWriter(StatsMetadata metadata) cfs.metadata, sstableMetadataCollector, SerializationHeader.make(cfs.metadata(), Sets.newHashSet(sstable)), - cfs.indexManager.listIndexes(), + cfs.indexManager.listIndexGroups(), transaction); } @@ -89,7 +88,7 @@ public void upgrade(boolean keepOriginals) outputHandler.output("Upgrading " + sstable); int nowInSec = FBUtilities.nowInSeconds(); try (SSTableRewriter writer = SSTableRewriter.construct(cfs, transaction, keepOriginals, CompactionTask.getMaxDataAge(transaction.originals())); - AbstractCompactionStrategy.ScannerList scanners = strategyManager.getScanners(transaction.originals()); + ScannerList scanners = strategyContainer.getScanners(transaction.originals()); CompactionIterator iter = new CompactionIterator(transaction.opType(), scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID())) { writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata())); diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java index 30e74adb4b35..f333438b40f8 100644 --- a/src/java/org/apache/cassandra/db/compaction/Verifier.java +++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java @@ -19,6 +19,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.cassandra.db.*; @@ -32,6 +33,7 @@ import org.apache.cassandra.io.sstable.IndexSummary; import org.apache.cassandra.io.sstable.KeyIterator; import org.apache.cassandra.io.sstable.SSTableIdentityIterator; +import org.apache.cassandra.io.sstable.format.PartitionIndexIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.metadata.MetadataComponent; import org.apache.cassandra.io.sstable.metadata.MetadataType; @@ -43,7 +45,7 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.BloomFilterSerializer; +import org.apache.cassandra.utils.BloomFilter; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.IFilter; @@ -76,9 +78,7 @@ public class Verifier implements Closeable private final ReadWriteLock fileAccessLock; private final RandomAccessReader dataFile; - private final RandomAccessReader indexFile; private final VerifyInfo verifyInfo; - private final RowIndexEntry.IndexSerializer rowIndexEntrySerializer; private final Options options; private final boolean isOffline; /** @@ -103,7 +103,6 @@ public Verifier(ColumnFamilyStore cfs, SSTableReader sstable, OutputHandler outp this.cfs = cfs; this.sstable = sstable; this.outputHandler = outputHandler; - this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(cfs.metadata(), sstable.descriptor.version, sstable.header); this.controller = new VerifyController(cfs); @@ -111,7 +110,6 @@ public Verifier(ColumnFamilyStore cfs, SSTableReader sstable, OutputHandler outp this.dataFile = isOffline ? sstable.openDataReader() : sstable.openDataReader(CompactionManager.instance.getRateLimiter()); - this.indexFile = RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))); this.verifyInfo = new VerifyInfo(dataFile, sstable, fileAccessLock.readLock()); this.options = options; this.isOffline = isOffline; @@ -149,7 +147,7 @@ public void verify() try { - outputHandler.debug("Deserializing index for "+sstable); + outputHandler.debug("Deserializing index for " + sstable); deserializeIndex(sstable); } catch (Throwable t) @@ -158,16 +156,19 @@ public void verify() markAndThrow(t); } - try - { - outputHandler.debug("Deserializing index summary for "+sstable); - deserializeIndexSummary(sstable); - } - catch (Throwable t) + if (sstable.descriptor.getFormat().supportedComponents().contains(Component.SUMMARY)) { - outputHandler.output("Index summary is corrupt - if it is removed it will get rebuilt on startup "+sstable.descriptor.filenameFor(Component.SUMMARY)); - outputHandler.warn(t); + try + { + outputHandler.debug("Deserializing index summary for " + sstable); + deserializeIndexSummary(sstable); + } + catch (Throwable t) + { + outputHandler.output("Index summary is corrupt - if it is removed it will get rebuilt on startup " + sstable.descriptor.filenameFor(Component.SUMMARY)); + outputHandler.warn(t); markAndThrow(t, false); + } } try @@ -185,7 +186,7 @@ public void verify() if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner)) { outputHandler.debug("Checking that all tokens are owned by the current node"); - try (KeyIterator iter = new KeyIterator(sstable.descriptor, sstable.metadata())) + try (KeyIterator iter = KeyIterator.forSSTable(sstable)) { List> ownedRanges = Range.normalize(tokenLookup.apply(cfs.metadata.keyspace)); if (ownedRanges.isEmpty()) @@ -239,14 +240,10 @@ public void verify() outputHandler.output("Extended Verify requested, proceeding to inspect values"); - try + try(PartitionIndexIterator indexIterator = sstable.allKeysIterator()) { - ByteBuffer nextIndexKey = ByteBufferUtil.readWithShortLength(indexFile); - { - long firstRowPositionFromIndex = rowIndexEntrySerializer.deserializePositionAndSkip(indexFile); - if (firstRowPositionFromIndex != 0) - markAndThrow(new RuntimeException("firstRowPositionFromIndex != 0: "+firstRowPositionFromIndex)); - } + if (indexIterator.dataPosition() != 0) + markAndThrow(new RuntimeException("First row position from index != 0: " + indexIterator.dataPosition())); List> ownedRanges = isOffline ? Collections.emptyList() : Range.normalize(tokenLookup.apply(cfs.metadata().keyspace)); RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges); @@ -256,7 +253,7 @@ public void verify() { if (verifyInfo.isStopRequested()) - throw new CompactionInterruptedException(verifyInfo.getCompactionInfo()); + throw new CompactionInterruptedException(verifyInfo.getProgress()); rowStart = dataFile.getFilePointer(); outputHandler.debug("Reading row at " + rowStart); @@ -285,14 +282,18 @@ public void verify() } } - ByteBuffer currentIndexKey = nextIndexKey; + ByteBuffer currentIndexKey = indexIterator.key(); long nextRowPositionFromIndex = 0; try { - nextIndexKey = indexFile.isEOF() ? null : ByteBufferUtil.readWithShortLength(indexFile); - nextRowPositionFromIndex = indexFile.isEOF() - ? dataFile.length() - : rowIndexEntrySerializer.deserializePositionAndSkip(indexFile); + if (indexIterator.advance()) + { + nextRowPositionFromIndex = indexIterator.dataPosition(); + } + else + { + nextRowPositionFromIndex = dataFile.length(); + } } catch (Throwable th) { @@ -309,8 +310,6 @@ public void verify() String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.getKey()); outputHandler.debug(String.format("row %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSize))); - assert currentIndexKey != null || indexFile.isEOF(); - try { if (key == null || dataSize > dataFile.length()) @@ -413,15 +412,12 @@ public boolean check(DecoratedKey key) private void deserializeIndex(SSTableReader sstable) throws IOException { - try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)))) - { - long indexSize = primaryIndex.length(); - - while ((primaryIndex.getFilePointer()) != indexSize) - { - ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex); - RowIndexEntry.Serializer.skip(primaryIndex, sstable.descriptor.version); - } + try (PartitionIndexIterator it = sstable.allKeysIterator()) { + //noinspection StatementWithEmptyBody + ByteBuffer last = it.key(); + while (it.advance()) last = it.key(); // no-op, just check if index is readable + if (!Objects.equals(last, sstable.last.getKey())) + throw new CorruptSSTableException(new IOException("Failed to read partition index"), it.toString()); } } @@ -448,7 +444,7 @@ private void deserializeBloomFilter(SSTableReader sstable) throws IOException if (Files.exists(bfPath)) { try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(bfPath))); - IFilter bf = BloomFilterSerializer.deserialize(stream, sstable.descriptor.version.hasOldBfFormat())) + IFilter bf = BloomFilter.serializer.deserialize(stream, sstable.descriptor.version.hasOldBfFormat())) { } } @@ -460,7 +456,6 @@ public void close() try { FileUtils.closeQuietly(dataFile); - FileUtils.closeQuietly(indexFile); } finally { @@ -486,7 +481,7 @@ private void markAndThrow(Throwable cause, boolean mutateRepaired) try { sstable.mutateRepairedAndReload(ActiveRepairService.UNREPAIRED_SSTABLE, sstable.getPendingRepair(), sstable.isTransient()); - cfs.getTracker().notifySSTableRepairedStatusChanged(Collections.singleton(sstable)); + cfs.getTracker().notifySSTableRepairedStatusChanged(ImmutableList.of(sstable)); } catch(IOException ioe) { @@ -500,12 +495,12 @@ private void markAndThrow(Throwable cause, boolean mutateRepaired) throw new RuntimeException(e); } - public CompactionInfo.Holder getVerifyInfo() + public AbstractTableOperation getVerifyInfo() { return verifyInfo; } - private static class VerifyInfo extends CompactionInfo.Holder + private static class VerifyInfo extends AbstractTableOperation { private final RandomAccessReader dataFile; private final SSTableReader sstable; @@ -520,17 +515,17 @@ public VerifyInfo(RandomAccessReader dataFile, SSTableReader sstable, Lock fileR verificationCompactionId = UUIDGen.getTimeUUID(); } - public CompactionInfo getCompactionInfo() + public OperationProgress getProgress() { fileReadLock.lock(); try { - return new CompactionInfo(sstable.metadata(), - OperationType.VERIFY, - dataFile.getFilePointer(), - dataFile.length(), - verificationCompactionId, - ImmutableSet.of(sstable)); + return new OperationProgress(sstable.metadata(), + OperationType.VERIFY, + dataFile.getFilePointer(), + dataFile.length(), + verificationCompactionId, + ImmutableSet.of(sstable)); } catch (Exception e) { diff --git a/src/java/org/apache/cassandra/db/compaction/unified/AdaptiveController.java b/src/java/org/apache/cassandra/db/compaction/unified/AdaptiveController.java new file mode 100644 index 000000000000..43cedcb9ba33 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/AdaptiveController.java @@ -0,0 +1,344 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MonotonicClock; + +/** + * The adaptive compaction controller dynamically calculates the optimal scaling parameter W. + *

    + * Generally it tries to find a local minimum for the total IO cost that is projected + * by the strategy. The projected IO cost is composed by two parts: the read amplification, + * which is weighted by the number of partitions read by the user, and the write amplification, which + * is weighted by the number of bytes inserted into memtables. Other parameters are also considered, such + * as the cache miss rate and the time it takes to read and write from disk. See also the comments in + * {@link CostsCalculator}. + * + * Design doc: TODO: link to design doc or SEP + */ +public class AdaptiveController extends Controller +{ + private static final Logger logger = LoggerFactory.getLogger(AdaptiveController.class); + + /** The starting value for the scaling parameter */ + static final String STARTING_SCALING_PARAMETER = "adaptive_starting_scaling_parameter"; + private static final int DEFAULT_STARTING_SCALING_PARAMETER = Integer.getInteger(PREFIX + STARTING_SCALING_PARAMETER, 0); + + /** The minimum valid value for the scaling parameter */ + static final String MIN_SCALING_PARAMETER = "adaptive_min_scaling_parameter"; + static private final int DEFAULT_MIN_SCALING_PARAMETER = Integer.getInteger(PREFIX + MIN_SCALING_PARAMETER, -10); + + /** The maximum valid value for the scaling parameter */ + static final String MAX_SCALING_PARAMETER = "adaptive_max_scaling_parameter"; + static private final int DEFAULT_MAX_SCALING_PARAMETER = Integer.getInteger(PREFIX + MIN_SCALING_PARAMETER, 36); + + /** The interval for periodically checking the optimal value for W */ + static final String INTERVAL_SEC = "adaptive_interval_sec"; + static private final int DEFAULT_INTERVAL_SEC = Integer.getInteger(PREFIX + INTERVAL_SEC, 300); + + /** The gain is a number between 0 and 1 used to determine if a new choice of W is better than the current one */ + static final String THRESHOLD = "adaptive_threshold"; + private static final double DEFAULT_THRESHOLD = Double.parseDouble(System.getProperty(PREFIX + THRESHOLD, "0.15")); + + /** Below the minimum cost we don't try to optimize W, we consider the current W good enough. This is necessary because the cost + * can vanish to zero when there are neither reads nor writes and right now we don't know how to handle this case. */ + static final String MIN_COST = "adaptive_min_cost"; + static private final int DEFAULT_MIN_COST = Integer.getInteger(PREFIX + MIN_COST, 1000); + + private final int intervalSec; + private final int minW; + private final int maxW; + private final double threshold; + private final int minCost; + + private volatile int W; + private volatile long lastChecked; + + @VisibleForTesting + public AdaptiveController(MonotonicClock clock, + Environment env, + int W, + double survivalFactor, + long dataSetSizeMB, + int numShards, + long minSstableSizeMB, + long flushSizeOverrideMB, + double maxSpaceOverhead, + int maxSSTablesToCompact, + long expiredSSTableCheckFrequency, + boolean ignoreOverlapsInExpirationCheck, + int intervalSec, + int minW, + int maxW, + double threshold, + int minCost) + { + super(clock, env, survivalFactor, dataSetSizeMB, numShards, minSstableSizeMB, flushSizeOverrideMB, maxSpaceOverhead, maxSSTablesToCompact, expiredSSTableCheckFrequency, ignoreOverlapsInExpirationCheck); + + this.W = W; + this.intervalSec = intervalSec; + this.minW = minW; + this.maxW = maxW; + this.threshold = threshold; + this.minCost = minCost; + } + + static Controller fromOptions(Environment env, + double survivalFactor, + long dataSetSizeMB, + int numShards, + long minSstableSizeMB, + long flushSizeOverrideMB, + double maxSpaceOverhead, + int maxSSTablesToCompact, + long expiredSSTableCheckFrequency, + boolean ignoreOverlapsInExpirationCheck, + Map options) + { + int W = options.containsKey(STARTING_SCALING_PARAMETER) ? Integer.parseInt(options.get(STARTING_SCALING_PARAMETER)) : DEFAULT_STARTING_SCALING_PARAMETER; + int minW = options.containsKey(MIN_SCALING_PARAMETER) ? Integer.parseInt(options.get(MIN_SCALING_PARAMETER)) : DEFAULT_MIN_SCALING_PARAMETER; + int maxW = options.containsKey(MAX_SCALING_PARAMETER) ? Integer.parseInt(options.get(MAX_SCALING_PARAMETER)) : DEFAULT_MAX_SCALING_PARAMETER; + int intervalSec = options.containsKey(INTERVAL_SEC) ? Integer.parseInt(options.get(INTERVAL_SEC)) : DEFAULT_INTERVAL_SEC; + double threshold = options.containsKey(THRESHOLD) ? Double.parseDouble(options.get(THRESHOLD)) : DEFAULT_THRESHOLD; + int minCost = options.containsKey(MIN_COST) ? Integer.parseInt(options.get(MIN_COST)) : DEFAULT_MIN_COST; + + return new AdaptiveController(MonotonicClock.preciseTime, env, W, survivalFactor, dataSetSizeMB, numShards, minSstableSizeMB, flushSizeOverrideMB, maxSpaceOverhead, maxSSTablesToCompact, expiredSSTableCheckFrequency, ignoreOverlapsInExpirationCheck, intervalSec, minW, maxW, threshold, minCost); + } + + public static Map validateOptions(Map options) throws ConfigurationException + { + int W = DEFAULT_STARTING_SCALING_PARAMETER; + int minW = DEFAULT_MIN_SCALING_PARAMETER; + int maxW = DEFAULT_MAX_SCALING_PARAMETER; + + String s; + s = options.remove(STARTING_SCALING_PARAMETER); + if (s != null) + W = Integer.parseInt(s); + s = options.remove(MIN_SCALING_PARAMETER); + if (s != null) + minW = Integer.parseInt(s); + s = options.remove(MAX_SCALING_PARAMETER); + if (s != null) + maxW = Integer.parseInt(s); + + if (minW >= maxW || W < minW || W > maxW) + throw new ConfigurationException(String.format("Invalid configuration for W: %d, min: %d, max: %d", W, minW, maxW)); + + s = options.remove(INTERVAL_SEC); + if (s != null) + { + int intervalSec = Integer.parseInt(s); + if (intervalSec <= 0) + throw new ConfigurationException(String.format("Invalid configuration for interval, it should be positive: %d", intervalSec)); + } + s = options.remove(THRESHOLD); + if (s != null) + { + double threshold = Double.parseDouble(s); + if (threshold <= 0 || threshold > 1) + { + throw new ConfigurationException(String.format("Invalid configuration for threshold, it should be within (0,1]: %f", threshold)); + } + } + s = options.remove(MIN_COST); + if (s != null) + { + int minCost = Integer.parseInt(s); + if (minCost <= 0) + throw new ConfigurationException(String.format("Invalid configuration for minCost, it should be positive: %d", minCost)); + } + return options; + } + + @Override + void startup(UnifiedCompactionStrategy strategy, CostsCalculator calculator) + { + super.startup(strategy, calculator); + this.lastChecked = clock.now(); + } + + @Override + public int getScalingParameter(int index) + { + return W; + } + + @Override + public double getSurvivalFactor() + { + return survivalFactor; + } + + @Override + @Nullable + public CostsCalculator getCalculator() + { + return calculator; + } + + public int getInterval() + { + return intervalSec; + } + + public int getMinW() + { + return minW; + } + + public int getMaxW() + { + return maxW; + } + + public double getThreshold() + { + return threshold; + } + + public int getMinCost() + { + return minCost; + } + + @Override + public void onStrategyBackgroundTaskRequest() + { + if (!isRunning()) + return; + + long now = clock.now(); + if (now - lastChecked < TimeUnit.SECONDS.toNanos(intervalSec)) + return; + + try + { + maybeUpdate(now); + } + finally + { + lastChecked = now; + } + } + + /** + * Maybe updates the scaling parameter according to the data size, read, and write costs. + * + * The scaling parameter calculation is based on current read and write query costs for the entire data size. + * We use the entire data size instead of shard size here because query cost calculations do not take + * sharding into account. Also, the same scaling parameter is going to be used across all shards. + * + * @param now current timestamp only used for debug logging + */ + private void maybeUpdate(long now) + { + final long targetSize = Math.max(getDataSetSizeBytes(), (long) Math.ceil(calculator.spaceUsed())); + + final int RA = readAmplification(targetSize, W); + final int WA = writeAmplification(targetSize, W); + + final double readCost = calculator.getReadCostForQueries(RA); + final double writeCost = calculator.getWriteCostForQueries(WA); + final double cost = readCost + writeCost; + + if (cost <= minCost) + { + logger.debug("Adaptive compaction controller not updated, cost for current W {} is below minimum cost {}: read cost: {}, write cost: {}\\nAverages: {}", W, minCost, readCost, writeCost, calculator); + return; + } + + final double[] totCosts = new double[maxW - minW + 1]; + final double[] readCosts = new double[maxW - minW + 1]; + final double[] writeCosts = new double[maxW - minW + 1]; + int candW = W; + double candCost = cost; + + for (int i = minW; i <= maxW; i++) + { + final int idx = i - minW; + if (i == W) + { + readCosts[idx] = readCost; + writeCosts[idx] = writeCost; + } + else + { + final int ra = readAmplification(targetSize, i); + final int wa = writeAmplification(targetSize, i); + + readCosts[idx] = calculator.getReadCostForQueries(ra); + writeCosts[idx] = calculator.getWriteCostForQueries(wa); + } + totCosts[idx] = readCosts[idx] + writeCosts[idx]; + // in case of a tie, for neg.ve Ws we prefer higher Ws (smaller WA), but not for pos.ve Ws we prefer lower Ws (more parallelism) + if (totCosts[idx] < candCost || (i < 0 && totCosts[idx] == candCost)) + { + candW = i; + candCost = totCosts[idx]; + } + } + + logger.debug("Min cost: {}, min W: {}, min sstable size: {}\nread costs: {}\nwrite costs: {}\ntot costs: {}\nAverages: {}", + candCost, + candW, + FBUtilities.prettyPrintMemory(getMinSstableSizeBytes()), + Arrays.toString(readCosts), + Arrays.toString(writeCosts), + Arrays.toString(totCosts), + calculator); + + StringBuilder str = new StringBuilder(100); + str.append("Adaptive compaction controller "); + + if (W != candW && (cost - candCost) >= threshold * cost) + { + str.append("updated ").append(W).append(" -> ").append(candW); + this.W = candW; + } + else + { + str.append("unchanged"); + } + + str.append(", data size: ").append(FBUtilities.prettyPrintMemory(targetSize)); + str.append(", query cost: ").append(cost); + str.append(", new query cost: ").append(candCost); + str.append(", took ").append(TimeUnit.NANOSECONDS.toMicros(clock.now() - now)).append(" us"); + + logger.debug(str.toString()); + } + + @Override + public String toString() + { + return String.format("m: %d, o: %f, W: %d - %s", minSstableSizeMB, survivalFactor, W, calculator); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/unified/Controller.java b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java new file mode 100644 index 000000000000..8cb060e182a6 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/Controller.java @@ -0,0 +1,832 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.cassandra.config.Config; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Gauge; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.compaction.CompactionStrategy; +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.metrics.DefaultNameFactory; +import org.apache.cassandra.metrics.MetricNameFactory; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.MonotonicClock; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +/** +* The controller provides compaction parameters to the unified compaction strategy +*/ +public abstract class Controller +{ + protected static final Logger logger = LoggerFactory.getLogger(Controller.class); + private static final ConcurrentMap allMetrics = new ConcurrentHashMap<>(); + + static final String PREFIX = "unified_compaction."; + + /** The data size in GB, it will be assumed that the node will have on disk roughly this size of data when it + * reaches equilibrium. By default 1 TB. */ + public static final String DATASET_SIZE_OPTION_GB = "dataset_size_in_gb"; + static final long DEFAULT_DATASET_SIZE_GB = Long.getLong(PREFIX + DATASET_SIZE_OPTION_GB, + DatabaseDescriptor.getDataFileDirectoriesMinTotalSpaceInGB()); + + /** The number of shards. The shard size will be calculated by dividing the data size by this number. + * By default, 10 would be used for single disk. If the data size is 1 TB, then the shard size becomes 100 GB. + * If JBOD / multi-drive, it would be 10 * disks. For example, if there are 5 disks, there would be 50 shards. + * With data size 10 TB, the shard size would be 200 GB. + * */ + static final String NUM_SHARDS_OPTION = "num_shards"; + static final int DEFAULT_NUM_SHARDS = Integer.getInteger(PREFIX + NUM_SHARDS_OPTION, + 10 * DatabaseDescriptor.getAllDataFileLocations().length); + + /** + * The minimum sstable size. Sharded writers split sstables over shard only if they are at least as large + * as the minimum size. + * + * When the minimum sstable size is zero in the compaction options, then it is calculated by the controller by + * looking at the initial flush size. + */ + static final String MIN_SSTABLE_SIZE_OPTION_MB = "min_sstable_size_in_mb"; + static final int DEFAULT_MIN_SSTABLE_SIZE_MB = Integer.getInteger(PREFIX + MIN_SSTABLE_SIZE_OPTION_MB, 100); + + /** + * Override for the flush size in MB. The database should be able to calculate this from executing flushes, this + * should only be necessary in rare cases. + */ + static final String FLUSH_SIZE_OVERRIDE_OPTION_MB = "flush_size_override_mb"; + + /** + * The maximum tolerable compaction-induced space amplification, as fraction of the dataset size. The idea behind + * this property is to be able to tune how much to limit concurrent "oversized" compactions in different shards. + * On one hand allowing such compactions concurrently running in all shards allows for STCS-like space + * amplification, where at some point you might need free space double the size of your working set to do a (top + * tier) compaction, while on the other hand limiting such compactions too much might lead to compaction lagging + * behind, higher read amplification, and other problems of that nature. + */ + static public final String MAX_SPACE_OVERHEAD_OPTION = "max_space_overhead"; + static final double DEFAULT_MAX_SPACE_OVERHEAD = Double.parseDouble(System.getProperty(PREFIX + MAX_SPACE_OVERHEAD_OPTION, "0.2")); + static final double MAX_SPACE_OVERHEAD_LOWER_BOUND = 0.01; + static final double MAX_SPACE_OVERHEAD_UPPER_BOUND = 1.0; + + /** + * This parameter is intended to modify the shape of the LSM by taking into account the survival ratio of data, for now it is fixed to one. + */ + static final double DEFAULT_SURVIVAL_FACTOR = Double.parseDouble(System.getProperty(PREFIX + "survival_factor", "1")); + + /** + * Either true or false. This parameter determines which controller will be used. + */ + static final String ADAPTIVE_OPTION = "adaptive"; + static final boolean DEFAULT_ADAPTIVE = Boolean.parseBoolean(System.getProperty(PREFIX + ADAPTIVE_OPTION, "false")); + + /** + * The maximum number of sstables to compact in one operation. + * + * This is expected to be large and never be reached, but compaction going very very late may cause the accumulation + * of thousands and even tens of thousands of sstables which may cause problems if compacted in one long operation. + * The default is chosen to be half of the maximum permitted space overhead when the source sstables are of the + * minimum sstable size. + * + * If the fanout factor is larger than the maximum number of sstables, the strategy will ignore the latter. + */ + static final String MAX_SSTABLES_TO_COMPACT_OPTION = "max_sstables_to_compact"; + + static final String ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION = "unsafe_aggressive_sstable_expiration"; + static final String ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_PROPERTY = Config.PROPERTY_PREFIX + "allow_unsafe_aggressive_sstable_expiration"; + static final boolean ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION = Boolean.parseBoolean(System.getProperty(ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_PROPERTY)); + static final boolean DEFAULT_ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION = false; + + static final int DEFAULT_EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS = 60 * 10; + static final String EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_OPTION = "expired_sstable_check_frequency_seconds"; + + protected final MonotonicClock clock; + protected final Environment env; + protected final double survivalFactor; + protected final long dataSetSizeMB; + protected final int numShards; + protected final long shardSizeMB; + protected volatile long minSstableSizeMB; + protected final double maxSpaceOverhead; + protected final long flushSizeOverrideMB; + protected volatile long currentFlushSize; + protected final int maxSSTablesToCompact; + protected final long expiredSSTableCheckFrequency; + protected final boolean ignoreOverlapsInExpirationCheck; + @Nullable protected volatile CostsCalculator calculator; + @Nullable private volatile Metrics metrics; + + Controller(MonotonicClock clock, + Environment env, + double survivalFactor, + long dataSetSizeMB, + int numShards, + long minSstableSizeMB, + long flushSizeOverrideMB, + double maxSpaceOverhead, + int maxSSTablesToCompact, + long expiredSSTableCheckFrequency, + boolean ignoreOverlapsInExpirationCheck) + { + this.clock = clock; + this.env = env; + this.survivalFactor = survivalFactor; + this.dataSetSizeMB = dataSetSizeMB; + this.numShards = numShards; + this.shardSizeMB = (int) Math.ceil((double) dataSetSizeMB / numShards); + this.minSstableSizeMB = minSstableSizeMB; + this.flushSizeOverrideMB = flushSizeOverrideMB; + this.currentFlushSize = flushSizeOverrideMB << 20; + this.expiredSSTableCheckFrequency = TimeUnit.MILLISECONDS.convert(expiredSSTableCheckFrequency, TimeUnit.SECONDS); + + double maxSpaceOverheadLowerBound = 1.0d / numShards; + if (maxSpaceOverhead < maxSpaceOverheadLowerBound) + { + logger.warn("{} shards are not enough to maintain the required maximum space overhead of {}!\n" + + "Falling back to {}={} instead. If this limit needs to be satisfied, please increase the number" + + " of shards.", + numShards, + maxSpaceOverhead, + MAX_SPACE_OVERHEAD_OPTION, + String.format("%.3f", maxSpaceOverheadLowerBound)); + this.maxSpaceOverhead = maxSpaceOverheadLowerBound; + } + else + this.maxSpaceOverhead = maxSpaceOverhead; + + if (maxSSTablesToCompact <= 0) // use half the maximum permitted compaction size as upper bound by default + maxSSTablesToCompact = (int) (dataSetSizeMB * this.maxSpaceOverhead * 0.5 / minSstableSizeMB); + + this.maxSSTablesToCompact = maxSSTablesToCompact; + + if (ignoreOverlapsInExpirationCheck && !ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION) + { + logger.warn("Not enabling aggressive SSTable expiration, as the system property '" + ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_PROPERTY + "' is set to 'false'. " + + "Set it to 'true' to enable aggressive SSTable expiration."); + } + this.ignoreOverlapsInExpirationCheck = ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION && ignoreOverlapsInExpirationCheck; + } + + @VisibleForTesting + public Environment getEnv() + { + return env; + } + + /** + * @return the scaling parameter W + * @param index + */ + public abstract int getScalingParameter(int index); + + public int getFanout(int index) { + int W = getScalingParameter(index); + return W < 0 ? 2 - W : 2 + W; // see formula in design doc + } + + public int getThreshold(int index) { + int W = getScalingParameter(index); + return W < 0 ? 2 : getFanout(index); // see formula in design doc + } + + /** + * @return the number of shards according to the dataset and shard sizes set by the user + */ + public int getNumShards() + { + return numShards; + } + + /** + * @return the survival factor o + */ + public double getSurvivalFactor() + { + return survivalFactor; + } + + /** + * The user specified dataset size. + * + * @return the target size of the entire data set, in bytes. + */ + public long getDataSetSizeBytes() + { + return dataSetSizeMB << 20; + } + + /** + * The user specified shard, or compaction arena, size. + * + * @return the desired size of each shard, or compaction arena, in bytes. + */ + public long getShardSizeBytes() + { + return shardSizeMB << 20; + } + + /** + * Return the sstable size in bytes. + * + * This is either set by the user in the options or calculated by rounding up the first flush size to 50 MB. + * + * @return the minimum sstable size in bytes. + */ + public long getMinSstableSizeBytes() + { + if (minSstableSizeMB > 0) + return minSstableSizeMB << 20; + + synchronized (this) + { + if (minSstableSizeMB > 0) + return minSstableSizeMB << 20; + + // round the avg flush size to the nearest byte + long envFlushSize = Math.round(env.flushSize()); + long fiftyMB = 50 << 20; + + // round up to 50 MB + long flushSize = ((Math.max(1, envFlushSize) + fiftyMB - 1) / fiftyMB) * fiftyMB; + + // If the env flush size is positive, then we've flushed at least once and we use this value permanently + if (envFlushSize > 0) + minSstableSizeMB = flushSize >> 20; + + return flushSize; + } + } + + /** + * Return the flush sstable size in bytes. + * + * This is usually obtained from the observed sstable flush sizes, refreshed when it differs significantly + * from the current values. + * It can also be set by the user in the options. + * + * @return the flush size in bytes. + */ + public long getFlushSizeBytes() + { + if (flushSizeOverrideMB > 0) + return flushSizeOverrideMB << 20; + + double envFlushSize = env.flushSize(); + if (currentFlushSize == 0 || Math.abs(1 - (currentFlushSize / envFlushSize)) > 0.5) + { + // The current size is not initialized, or it differs by over 50% from the observed. + // Use the observed size rounded up to a whole megabyte. + currentFlushSize = ((long) (Math.ceil(Math.scalb(envFlushSize, -20)))) << 20; + } + return currentFlushSize; + } + + /** + * Returns the maximum tolerable compaction-induced space amplification, as a fraction of the dataset size. + * Currently this is not a strict limit for which compaction gives an ironclad guarantee never to exceed it, but + * the main input in a simple heuristic that is designed to limit UCS' space amplification in exchange of some + * delay in top bucket compactions. + * + * @return a {@code double} value between 0.01 and 1.0, representing the fraction of the expected uncompacted + * dataset size that should be additionally available for compaction's space amplification overhead. + */ + public double getMaxSpaceOverhead() + { + return maxSpaceOverhead; + } + + /** + * @return whether is allowed to drop expired SSTables without checking if partition keys appear in other SSTables. + * Same behavior as in TWCS. + */ + public boolean getIgnoreOverlapsInExpirationCheck() + { + return ignoreOverlapsInExpirationCheck; + } + + public long getExpiredSSTableCheckFrequency() + { + return expiredSSTableCheckFrequency; + } + + /** + * Perform any initialization that requires the strategy. + */ + public void startup(UnifiedCompactionStrategy strategy, ScheduledExecutorService executorService) + { + if (calculator != null) + throw new IllegalStateException("Already started"); + + startup(strategy, new CostsCalculator(env, strategy, executorService, survivalFactor)); + } + + @VisibleForTesting + void startup(UnifiedCompactionStrategy strategy, CostsCalculator calculator) + { + this.calculator = calculator; + metrics = allMetrics.computeIfAbsent(strategy.getMetadata(), Controller.Metrics::new); + metrics.setController(this); + logger.debug("Started compaction controller {}", this); + } + + /** + * Signals that the strategy is about to be deleted or stopped. + */ + public void shutdown() + { + if (calculator == null) + return; + + calculator.close(); + calculator = null; + + if (metrics != null) + { + metrics.release(); + metrics.removeController(); + metrics = null; + } + + logger.debug("Stopped compaction controller {}", this); + } + + /** + * @return true if the controller is running + */ + public boolean isRunning() + { + return calculator != null; + } + + /** + * @return the cost calculator, will be null until {@link this#startup(UnifiedCompactionStrategy, ScheduledExecutorService)} is called. + */ + @Nullable + @VisibleForTesting + public CostsCalculator getCalculator() + { + return calculator; + } + + /** + * The strategy will call this method each time {@link CompactionStrategy#getNextBackgroundTasks(int)} is called. + */ + public void onStrategyBackgroundTaskRequest() + { + } + + /** + * Calculate the read amplification assuming a single scaling parameter W and a given total + * length of data on disk. + * + * @param length the total length on disk + * @param scalingParameter the scaling parameter to use for the calculation + * + * @return the read amplification of all the buckets needed to cover the total length + */ + public int readAmplification(long length, int scalingParameter) + { + double o = getSurvivalFactor(); + long m = getFlushSizeBytes(); + + int F = scalingParameter < 0 ? 2 - scalingParameter : 2 + scalingParameter; + int T = scalingParameter < 0 ? 2 : F; + int maxIndex = maxBucketIndex(length, F); + + int ret = 0; + for (int i = 0; i < maxIndex; i++) + ret += T - 1; + + if (scalingParameter >= 0) + ret += Math.max(0, Math.ceil(length / (m * Math.pow(o * F, maxIndex))) - 1); + else + ret += 1; + + return ret; + } + + /** + * Calculate the write amplification assuming a single scaling parameter W and a given total + * length of data on disk. + * + * @param length the total length on disk + * @param scalingParameter the scaling parameter to use for the calculation + * + * @return the write amplification of all the buckets needed to cover the total length + */ + public int writeAmplification(long length, int scalingParameter) + { + double o = getSurvivalFactor(); + long m = getFlushSizeBytes(); + + int F = scalingParameter < 0 ? 2 - scalingParameter : 2 + scalingParameter; + int maxIndex = maxBucketIndex(length, F); + + int ret = 0; + + if (scalingParameter >= 0) + { // for tiered, at each level the WA is 1. We start at level 0 and end up at level maxIndex so that's a WA of maxIndex. + ret += maxIndex + 1; + } + else + { // for leveled, at each level the WA is F - 1 except for the last one, where it's (size / size of previous level) - 1 + // or (size / (m*(o*F)^maxIndex)) - 1 + for (int i = 0; i < maxIndex; i++) + ret += F - 1; + + ret += Math.max(0, Math.ceil(length / (m * Math.pow(o * F, maxIndex)))); + } + + return ret; + } + + /** + * Returns a maximum bucket index for the given data size and fanout. + */ + private int maxBucketIndex(long totalLength, int fanout) + { + double o = getSurvivalFactor(); + long m = getFlushSizeBytes(); + return Math.max(0, (int) Math.floor((Math.log(totalLength) - Math.log(m)) / (Math.log(fanout) - Math.log(o)))); + } + + private double getReadIOCost() + { + if (calculator == null) + return 0; + + int W = getScalingParameter(0); + long length = (long) Math.ceil(calculator.spaceUsed()); + return calculator.getReadCostForQueries(readAmplification(length, W)); + } + + private double getWriteIOCost() + { + if (calculator == null) + return 0; + + int W = getScalingParameter(0); + long length = (long) Math.ceil(calculator.spaceUsed()); + return calculator.getWriteCostForQueries(writeAmplification(length, W)); + } + + public static Controller fromOptions(ColumnFamilyStore cfs, Map options) + { + boolean adaptive = options.containsKey(ADAPTIVE_OPTION) ? Boolean.parseBoolean(options.get(ADAPTIVE_OPTION)) : DEFAULT_ADAPTIVE; + long dataSetSizeMb = (options.containsKey(DATASET_SIZE_OPTION_GB) ? Long.parseLong(options.get(DATASET_SIZE_OPTION_GB)) : DEFAULT_DATASET_SIZE_GB) << 10; + int numShards = options.containsKey(NUM_SHARDS_OPTION) ? Integer.parseInt(options.get(NUM_SHARDS_OPTION)) : DEFAULT_NUM_SHARDS; + long sstableSizeMb = options.containsKey(MIN_SSTABLE_SIZE_OPTION_MB) ? Long.parseLong(options.get(MIN_SSTABLE_SIZE_OPTION_MB)) : DEFAULT_MIN_SSTABLE_SIZE_MB; + long flushSizeOverrideMb = Long.parseLong(options.getOrDefault(FLUSH_SIZE_OVERRIDE_OPTION_MB, "0")); + double maxSpaceOverhead = options.containsKey(MAX_SPACE_OVERHEAD_OPTION) + ? Double.parseDouble(options.get(MAX_SPACE_OVERHEAD_OPTION)) + : DEFAULT_MAX_SPACE_OVERHEAD; + int maxSSTablesToCompact = Integer.parseInt(options.getOrDefault(MAX_SSTABLES_TO_COMPACT_OPTION, "0")); + long expiredSSTableCheckFrequency = options.containsKey(EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_OPTION) + ? Long.parseLong(options.get(EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_OPTION)) + : DEFAULT_EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS; + boolean ignoreOverlapsInExpirationCheck = options.containsKey(ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION) + ? Boolean.parseBoolean(options.get(ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION)) + : DEFAULT_ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION; + + Environment env = new RealEnvironment(cfs); + + return adaptive + ? AdaptiveController.fromOptions(env, + DEFAULT_SURVIVAL_FACTOR, + dataSetSizeMb, + numShards, + sstableSizeMb, + flushSizeOverrideMb, + maxSpaceOverhead, + maxSSTablesToCompact, + expiredSSTableCheckFrequency, + ignoreOverlapsInExpirationCheck, + options) + : StaticController.fromOptions(env, + DEFAULT_SURVIVAL_FACTOR, + dataSetSizeMb, + numShards, + sstableSizeMb, + flushSizeOverrideMb, + maxSpaceOverhead, + maxSSTablesToCompact, + expiredSSTableCheckFrequency, + ignoreOverlapsInExpirationCheck, + options); + } + + public static Map validateOptions(Map options) throws ConfigurationException + { + String nonPositiveErr = "Invalid configuration, %s should be positive: %d"; + String booleanParseErr = "%s should either be 'true' or 'false', not %s"; + String intParseErr = "%s is not a parsable int (base10) for %s"; + String longParseErr = "%s is not a parsable long (base10) for %s"; + String floatParseErr = "%s is not a parsable float for %s"; + options = new HashMap<>(options); + String s; + boolean adaptive = DEFAULT_ADAPTIVE; + + s = options.remove(ADAPTIVE_OPTION); + if (s != null) + { + if (!s.equalsIgnoreCase("true") && !s.equalsIgnoreCase("false")) + { + throw new ConfigurationException(String.format(booleanParseErr, ADAPTIVE_OPTION, s)); + } + adaptive = Boolean.parseBoolean(s); + } + + s = options.remove(MIN_SSTABLE_SIZE_OPTION_MB); + if (s != null) + { + try + { + long minSStableSize = Long.parseLong(s); + if (minSStableSize <= 0) + throw new ConfigurationException(String.format(nonPositiveErr, + MIN_SSTABLE_SIZE_OPTION_MB, + minSStableSize)); + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format(longParseErr, + s, + MIN_SSTABLE_SIZE_OPTION_MB), + e); + } + } + + s = options.remove(FLUSH_SIZE_OVERRIDE_OPTION_MB); + if (s != null) + { + try + { + long flushSize = Long.parseLong(s); + if (flushSize <= 0) + throw new ConfigurationException(String.format(nonPositiveErr, + FLUSH_SIZE_OVERRIDE_OPTION_MB, + flushSize)); + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format(longParseErr, + s, + FLUSH_SIZE_OVERRIDE_OPTION_MB), + e); + } + } + + s = options.remove(DATASET_SIZE_OPTION_GB); + if (s != null) + { + try + { + long dataSetSizeMb = Long.parseLong(s); + if (dataSetSizeMb <= 0) + throw new ConfigurationException(String.format(nonPositiveErr, + DATASET_SIZE_OPTION_GB, + dataSetSizeMb)); + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format(longParseErr, s, DATASET_SIZE_OPTION_GB), e); + } + } + + s = options.remove(NUM_SHARDS_OPTION); + if (s != null) + { + try + { + int numShards = Integer.parseInt(s); + if (numShards <= 0) + throw new ConfigurationException(String.format(nonPositiveErr, + NUM_SHARDS_OPTION, + numShards)); + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format(intParseErr, s, NUM_SHARDS_OPTION), e); + } + } + s = options.remove(MAX_SSTABLES_TO_COMPACT_OPTION); + if (s != null) + { + try + { + Integer.parseInt(s); // values less than or equal to 0 enable the default + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format(intParseErr, + s, + MAX_SSTABLES_TO_COMPACT_OPTION), + e); + } + } + s = options.remove(EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_OPTION); + if (s != null) + { + try + { + long expiredSSTableCheckFrequency = Long.parseLong(s); + if (expiredSSTableCheckFrequency <= 0) + throw new ConfigurationException(String.format(nonPositiveErr, + EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_OPTION, + expiredSSTableCheckFrequency)); + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format(longParseErr, + s, + EXPIRED_SSTABLE_CHECK_FREQUENCY_SECONDS_OPTION), + e); + } + } + + s = options.remove(MAX_SPACE_OVERHEAD_OPTION); + if (s != null) + { + try + { + double maxSpaceOverhead = Double.parseDouble(s); + if (maxSpaceOverhead < MAX_SPACE_OVERHEAD_LOWER_BOUND || maxSpaceOverhead > MAX_SPACE_OVERHEAD_UPPER_BOUND) + throw new ConfigurationException(String.format("Invalid configuration, %s must be between %f and %f: %s", + MAX_SPACE_OVERHEAD_OPTION, + MAX_SPACE_OVERHEAD_LOWER_BOUND, + MAX_SPACE_OVERHEAD_UPPER_BOUND, + s)); + } + catch (NumberFormatException e) + { + throw new ConfigurationException(String.format(floatParseErr, + s, + MAX_SPACE_OVERHEAD_OPTION), + e); + } + } + + s = options.remove(ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION); + if (s != null && !s.equalsIgnoreCase("true") && !s.equalsIgnoreCase("false")) + { + throw new ConfigurationException(String.format(booleanParseErr, + ALLOW_UNSAFE_AGGRESSIVE_SSTABLE_EXPIRATION_OPTION, s)); + } + + return adaptive ? AdaptiveController.validateOptions(options) : StaticController.validateOptions(options); + } + + // The methods below are implemented here (rather than directly in UCS) to aid testability. + + public double getBaseSstableSize(int F) + { + // The compaction hierarchy should start at a minimum size which is close to the typical flush size, with + // some leeway to make sure we don't overcompact when flushes end up a little smaller. + // The leeway should be less than 1/F, though, to make sure we don't overshoot the boundary combining F-1 + // sources instead of F. + // Note that while we have not had flushes, the size will be 0 and we will use 1MB as the flush size. With + // fixed and positive W this should not hurt us, as the hierarchy will be in multiples of F and will still + // result in the same buckets, but for negative W or hybrid strategies this may cause temporary overcompaction. + // If this is a concern, the flush size override should be used to avoid it until DB-4401. + return Math.max(1 << 20, getFlushSizeBytes()) * (1.0 - 0.9 / F) / getNumShards(); + } + + public double maxThroughput() + { + final int compactionThroughputMbPerSec = DatabaseDescriptor.getCompactionThroughputMbPerSec(); + if (compactionThroughputMbPerSec <= 0) + return Double.MAX_VALUE; + return compactionThroughputMbPerSec * 1024.0 * 1024.0; + } + + public int maxConcurrentCompactions() + { + return DatabaseDescriptor.getConcurrentCompactors(); + } + + public long maxCompactionSpaceBytes() + { + // Note: Compaction will not proceed with operations larger than this size (i.e. it will compact on the lower + // levels but will accumulate sstables on the top until the space on the drive fills up). This sounds risky but + // is less of a problem than running out of space during compaction. + return (long) (getDataSetSizeBytes() * getMaxSpaceOverhead()); + } + + public int maxSSTablesToCompact() + { + return maxSSTablesToCompact; + } + + /** + * Random number generator to be used for the selection of tasks. + * Replaced by some tests. + */ + public Random random() + { + return ThreadLocalRandom.current(); + } + + static final class Metrics + { + private final MetricNameFactory factory; + private final AtomicReference controllerRef; + private final Gauge totWAGauge; + private final Gauge readIOCostGauge; + private final Gauge writeIOCostGauge; + private final Gauge totIOCostGauge; + + Metrics(TableMetadata metadata) + { + this.factory = new DefaultNameFactory("CompactionCosts", + String.format("%s.%s", metadata.keyspace, metadata.name)); + this.controllerRef = new AtomicReference<>(); + this.totWAGauge = Metrics.register(factory.createMetricName("WA"), this::getMeasuredWA); + this.readIOCostGauge = Metrics.register(factory.createMetricName("ReadIOCost"), this::getReadIOCost); + this.writeIOCostGauge = Metrics.register(factory.createMetricName("WriteIOCost"), this::getWriteIOCost); + this.totIOCostGauge = Metrics.register(factory.createMetricName("TotIOCost"), this::getTotalIOCost); + } + + void setController(Controller controller) + { + this.controllerRef.set(controller); + } + + void removeController() + { + this.controllerRef.set(null); + } + + void release() + { + Metrics.remove(factory.createMetricName("WA")); + Metrics.remove(factory.createMetricName("ReadIOCost")); + Metrics.remove(factory.createMetricName("WriteIOCost")); + Metrics.remove(factory.createMetricName("TotIOCost")); + } + + double getMeasuredWA() + { + double ret = 0; + Controller controller = controllerRef.get(); + if (controller != null) + ret = controller.env.WA(); + + return ret; + } + + double getReadIOCost() + { + double ret = 0; + Controller controller = controllerRef.get(); + if (controller != null) + ret = controller.getReadIOCost(); + + return ret; + } + + double getWriteIOCost() + { + double ret = 0; + Controller controller = controllerRef.get(); + if (controller != null) + ret = controller.getWriteIOCost(); + + return ret; + } + + double getTotalIOCost() + { + return getReadIOCost() + getWriteIOCost(); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/unified/CostsCalculator.java b/src/java/org/apache/cassandra/db/compaction/unified/CostsCalculator.java new file mode 100644 index 000000000000..673d4b77dd17 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/CostsCalculator.java @@ -0,0 +1,277 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.metrics.CompactionMetrics; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.MovingAverage; + +/** + * This class periodically retrieves delta values from the environment and stores them into exponentially weighted averages. + * It then uses these values to calculate IO costs that are exported to {@link CompactionMetrics} and used by {@link AdaptiveController} + * to choose the optimal configuration for compaction. + */ +public class CostsCalculator +{ + private final static Logger logger = LoggerFactory.getLogger(CostsCalculator.class); + + /** How often values are sampled. Sampling for periods that are too short (<= 1 second) may not give good results since + * we many not collect sufficient data. */ + final static int samplingPeriodMs = Integer.getInteger(Controller.PREFIX + "sample_time_ms", 5000); + + /** The multipliers can be used by users if they wish to adjust the costs. We reduce the read costs because writes are batch processes (flush and compaction) + * and therefore the costs tend to be lower that for reads, so by reducing read costs we make the costs more comparable. + */ + final static double defaultWriteMultiplier = Double.parseDouble(System.getProperty(Controller.PREFIX + "costs_write_multiplier", "1")); + final static double defaultReadMultiplier = Double.parseDouble(System.getProperty(Controller.PREFIX + "costs_read_multiplier", "0.1")); + + private final Environment env; + private final double readMultiplier; + private final double writeMultiplier; + private final double survivalFactor; + private final MovingAverageOfDelta partitionsReadPerPeriod; + private final MovingAverageOfDelta bytesInsertedPerPeriod; + private final MovingAverage numSSTables; + private final MovingAverage spaceUsed; + private final UnifiedCompactionStrategy strategy; + + private final ReentrantReadWriteLock lock; + private final ReentrantReadWriteLock.ReadLock readLock; + private final ReentrantReadWriteLock.WriteLock writeLock; + private final ScheduledFuture future; + + CostsCalculator(Environment env, + UnifiedCompactionStrategy strategy, + ScheduledExecutorService executorService, + double survivalFactor) + { + this(env, strategy, executorService, survivalFactor, defaultReadMultiplier, defaultWriteMultiplier); + } + + CostsCalculator(Environment env, + UnifiedCompactionStrategy strategy, + ScheduledExecutorService executorService, + double survivalFactor, + double readMultiplier, + double writeMultiplier) + { + this.env = env; + this.readMultiplier = readMultiplier; + this.writeMultiplier = writeMultiplier; + this.survivalFactor = survivalFactor; + this.partitionsReadPerPeriod = new MovingAverageOfDelta(env.makeExpMovAverage()); + this.bytesInsertedPerPeriod = new MovingAverageOfDelta(env.makeExpMovAverage()); + this.numSSTables = env.makeExpMovAverage(); + this.spaceUsed = env.makeExpMovAverage(); + this.strategy = strategy; + this.lock = new ReentrantReadWriteLock(); + this.readLock = lock.readLock(); + this.writeLock = lock.writeLock(); + this.future = executorService.scheduleAtFixedRate(this::sampleValues, samplingPeriodMs, samplingPeriodMs, TimeUnit.MILLISECONDS); + } + + public void close() + { + writeLock.lock(); + + try + { + logger.debug("Stopping cost calculations for {}", strategy.getMetadata()); + future.cancel(false); + logger.debug("Stopped cost calculations for {}", strategy.getMetadata()); + } + finally + { + writeLock.unlock(); + } + } + + @VisibleForTesting + void sampleValues() + { + writeLock.lock(); + + try + { + partitionsReadPerPeriod.update(env.partitionsRead()); + bytesInsertedPerPeriod.update(env.bytesInserted()); + + numSSTables.update(strategy.getSSTables().size()); + spaceUsed.update(strategy.getSSTables().stream().map(SSTableReader::onDiskLength).reduce(0L, Long::sum)); + } + catch (Throwable err) + { + JVMStabilityInspector.inspectThrowable(err); + logger.error("Failed to update values: {}/{}", err.getClass().getName(), err.getMessage(), err); + } + finally + { + writeLock.unlock(); + } + } + + /** + * @return the estimated read cost for the given number of partitions, in milliseconds + */ + private double getReadCost(double partitionsRead) + { + return (env.sstablePartitionReadLatencyNanos() * partitionsRead) / TimeUnit.MILLISECONDS.toNanos(1); + } + + /** + * Calculate the projected read cost for user queries. + * + * The projected read cost is given by the number of partitions read, times the mean partition latency and is calculated + * by {@link this#getReadCost(double)}. This value is then multiplied by the number of sstables we're likely to hit + * per partition read and the read multiplier. + *

    + * The number of sstables is calculated as Math.min(1 + env.bloomFilterFpRatio() * RA / survivalFactor, RA). Here we + * assume there is going to be at least one sstable accessed, possibly more in case of : + * + * - bloom filter's false positives; + * - partitions not surviving a compaction (1/survivalFactor is the limit of the sum of (1-survivalFactor)^n), that + * is partitions that would not exist if compaction was done; Note that the survival factor is currently fixed to 1. + * + * The RA is then a cap since we cannot read more than RA sstables, which are the sstables that exist because + * compactions allows them to exist. + *

    + * The read multiplier is a factor that operators can use to tweak the algorithm. + *

    + * @param RA the expected read amplification due to the current choice of compaction strategy + * + * @return the projected read cost for user queries + */ + public double getReadCostForQueries(int RA) + { + readLock.lock(); + + try + { + return getReadCost(partitionsReadPerPeriod.avg.get()) * Math.min(1 + env.bloomFilterFpRatio() * RA / survivalFactor, RA) * readMultiplier; + } + finally + { + readLock.unlock(); + } + } + + private double getFlushCost(double bytesWritten) + { + return ((bytesWritten / (1 << 10)) * env.flushLatencyPerKbInNanos()) / (double) TimeUnit.MILLISECONDS.toNanos(1); + } + + private double getCompactionCost(double bytesWritten) + { + // So, the compaction latency will depend on the size of the sstables, so in the correct solution each level + // should pass its output size and we should measure latency in MB or something like that + return ((bytesWritten / (1 << 10)) * env.compactionLatencyPerKbInNanos()) / (double) TimeUnit.MILLISECONDS.toNanos(1); + } + + /** + * Calculate the projected write cost for user insertions. + * + * The projected write cost is given by the number of bytes that were inserted times the flush cost + * plus the same number of bytes times the compaction cost and the compaction WA. We also multiply by + * a write multiplier to let users change the weights if needed. + * + * @param WA the expected write amplification due to compaction + * + * @return the projected flush and write cost. + */ + public double getWriteCostForQueries(int WA) + { + readLock.lock(); + + try + { + double bytesInserted = this.bytesInsertedPerPeriod.avg.get(); + // using bytesInserted for the compaction cost doesn't take into account overwrites but for now it's good enough + return (getFlushCost(bytesInserted) + getCompactionCost(bytesInserted) * WA) * writeMultiplier; + } + finally + { + readLock.unlock(); + } + } + + public double partitionsRead() + { + return partitionsReadPerPeriod.avg.get(); + } + + public double numSSTables() + { + return numSSTables.get(); + } + + public double spaceUsed() + { + return spaceUsed.get(); + } + + public Environment getEnv() + { + return env; + } + + @Override + public String toString() + { + return String.format("num partitions read %s, bytes inserted: %s, num sstables %s; Environment: %s", + partitionsReadPerPeriod, bytesInsertedPerPeriod, numSSTables, env); + } + + @NotThreadSafe + private static final class MovingAverageOfDelta + { + private final MovingAverage avg; + private volatile double prev; + + MovingAverageOfDelta(MovingAverage avg) + { + this.avg = avg; + this.prev = Double.MIN_VALUE; + } + + void update(double val) + { + if (prev != Double.MIN_VALUE) + avg.update(val - prev); + + prev = val; + } + + @Override + public String toString() + { + return String.format("%s/%d sec", FBUtilities.prettyPrintMemory((long) avg.get()), TimeUnit.MILLISECONDS.toSeconds(samplingPeriodMs)); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/unified/Environment.java b/src/java/org/apache/cassandra/db/compaction/unified/Environment.java new file mode 100644 index 000000000000..f94141222fa5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/Environment.java @@ -0,0 +1,82 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import org.apache.cassandra.utils.MovingAverage; + +/** + * This class supplies to the cost calculator the required parameters for the calculations. + * There are two implementations, one used in real life and one for the simulation. + */ +public interface Environment +{ + /** + * @return an exponential moving average. New values have greater representation in the average, and older samples' + * effect exponentially decays with new data. + */ + MovingAverage makeExpMovAverage(); + + /** + * @return the cache miss ratio in the last 5 minutes + */ + double cacheMissRatio(); + + /** + * @return the bloom filter false positive ratio for all sstables + */ + double bloomFilterFpRatio(); + + /** + * @return the size of the chunk that read from disk. + */ + int chunkSize(); + + /** + * @return the total bytes inserted into the memtables so far + */ + long bytesInserted(); + + /** + * @return the total number of partitions read so far + */ + long partitionsRead(); + + /** + * @return the mean read latency in nano seconds to read a partition from an sstable + */ + double sstablePartitionReadLatencyNanos(); + + /** + * @return the mean compaction time per 1 Kb of input, in nano seconds + */ + double compactionLatencyPerKbInNanos(); + + /** + * @return the mean flush latency per 1 Kb of input, in nano seconds + */ + double flushLatencyPerKbInNanos(); + + /** + * @return the write amplification (bytes flushed + bytes compacted / bytes flushed). + */ + double WA(); + + /** + * @return the average size of sstables when they are flushed, averaged over the last 5 minutes. + */ + double flushSize(); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/unified/RealEnvironment.java b/src/java/org/apache/cassandra/db/compaction/unified/RealEnvironment.java new file mode 100644 index 000000000000..ed98bfd23616 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/RealEnvironment.java @@ -0,0 +1,127 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.cache.ChunkCache; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.utils.ExpMovingAverage; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MovingAverage; +import org.apache.cassandra.utils.PageAware; + +/** + * An implementation of {@link Environment} that returns + * real values. + */ +class RealEnvironment implements Environment +{ + private final ColumnFamilyStore cfs; + + RealEnvironment(ColumnFamilyStore cfs) + { + this.cfs = cfs; + } + + @Override + public MovingAverage makeExpMovAverage() + { + return ExpMovingAverage.decayBy100(); + } + + @Override + public double cacheMissRatio() + { + double hitRate = ChunkCache.instance.metrics.hitRate.getValue(); + if (Double.isNaN(hitRate)) + return 1; // if the cache is not yet initialized then assume all requests are a cache miss + + return 1 - Math.min(1, hitRate); // hit rate should never be > 1 but just in case put a check + } + + @Override + public double bloomFilterFpRatio() + { + return cfs.bloomFilterFpRatio(); + } + + @Override + public int chunkSize() + { + CompressionParams compressionParams = cfs.metadata().params.compression; + if (compressionParams.isEnabled()) + return compressionParams.chunkLength(); + + return PageAware.PAGE_SIZE; + } + + @Override + public long partitionsRead() + { + return cfs.getReadRequests(); + } + + @Override + public double sstablePartitionReadLatencyNanos() + { + return cfs.sstablePartitionReadLatency(); + } + + @Override + public double compactionLatencyPerKbInNanos() + { + return cfs.getCompactionTimePerKb(); + } + + @Override + public double flushLatencyPerKbInNanos() + { + return cfs.getFlushTimePerKb(); + } + + @Override + public long bytesInserted() + { + return cfs.getBytesInserted(); + } + + @Override + public double WA() + { + return cfs.getWA(); + } + + @Override + public double flushSize() + { + return cfs.getFlushSizeOnDisk(); + } + + @Override + public String toString() + { + return String.format("Read latency: %d us / partition, flush latency: %d us / KiB, compaction latency: %d us / KiB, bfpr: %f, measured WA: %.2f, flush size %s", + TimeUnit.NANOSECONDS.toMicros((long) sstablePartitionReadLatencyNanos()), + TimeUnit.NANOSECONDS.toMicros((long) flushLatencyPerKbInNanos()), + TimeUnit.NANOSECONDS.toMicros((long) compactionLatencyPerKbInNanos()), + bloomFilterFpRatio(), + WA(), + FBUtilities.prettyPrintMemory((long)flushSize())); + } +} diff --git a/src/java/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriter.java new file mode 100644 index 000000000000..01a6be45d18c --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/ShardedCompactionWriter.java @@ -0,0 +1,142 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.compaction.ArenaSelector; +import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.utils.FBUtilities; + +/** + * A {@link CompactionAwareWriter} that splits the output sstable at the partition boundaries of the compaction + * shards used by {@link org.apache.cassandra.db.compaction.UnifiedCompactionStrategy} as long as the size of + * the sstable so far is sufficiently large. + */ +public class ShardedCompactionWriter extends CompactionAwareWriter +{ + protected final static Logger logger = LoggerFactory.getLogger(ShardedCompactionWriter.class); + + private final long minSstableSizeInBytes; + private final List boundaries; + private final double overwriteRatio; + + private int currentIndex; + + public ShardedCompactionWriter(ColumnFamilyStore cfs, + Directories directories, + LifecycleTransaction txn, + Set nonExpiredSSTables, + boolean keepOriginals, + long minSstableSizeInBytes, + List boundaries) + { + super(cfs, directories, txn, nonExpiredSSTables, keepOriginals); + + this.minSstableSizeInBytes = minSstableSizeInBytes; + this.boundaries = boundaries; + this.currentIndex = 0; + long totalKeyCount = nonExpiredSSTables.stream() + .mapToLong(SSTableReader::estimatedKeys) + .sum(); + this.overwriteRatio = 1.0 * SSTableReader.getApproximateKeyCount(nonExpiredSSTables) / totalKeyCount; + } + + @Override + protected boolean shouldSwitchWriterInCurrentLocation(DecoratedKey key) + { + boolean boundaryCrossed = false; + /* + The comparison to detect a boundary is costly, but if we only do this when the size is above the threshold, + we may detect a boundary change in the middle of a shard and split sstables at the wrong place. + */ + while (currentIndex < boundaries.size() && key.compareTo(boundaries.get(currentIndex)) >= 0) + { + currentIndex++; + boundaryCrossed = true; + } + + if (boundaryCrossed && sstableWriter.currentWriter().getEstimatedOnDiskBytesWritten() >= minSstableSizeInBytes) + { + logger.debug("Switching writer at boundary {}/{} index {}, with size {} for {}.{}", + key.getToken(), boundaries.get(currentIndex-1), currentIndex-1, + FBUtilities.prettyPrintMemory(sstableWriter.currentWriter().getEstimatedOnDiskBytesWritten()), + cfs.getKeyspaceName(), cfs.getTableName()); + return true; + } + + return false; + } + + @Override + @SuppressWarnings("resource") + protected SSTableWriter sstableWriter(Directories.DataDirectory directory, PartitionPosition diskBoundary) + { + while (diskBoundary != null && currentIndex < boundaries.size() && diskBoundary.compareTo(boundaries.get(currentIndex)) < 0) + currentIndex++; + + return SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(directory)), + shardAdjustedKeyCount(currentIndex, boundaries, minSstableSizeInBytes, nonExpiredSSTables, overwriteRatio), + minRepairedAt, + pendingRepair, + isTransient, + cfs.metadata, + new MetadataCollector(txn.originals(), cfs.metadata().comparator, 0), + SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), + cfs.indexManager.listIndexGroups(), + txn); + } + + private long shardAdjustedKeyCount(int shardIdx, + List boundaries, + long minSstableSizeInBytes, + Set sstables, + double overwriteRatio) + { + long shardAdjustedSize = 0; + long shardAdjustedKeyCount = 0; + for (int i = shardIdx; i < boundaries.size(); i++) + { + Set sstablesForShard = ArenaSelector.sstablesFor(i, boundaries, sstables); + for (SSTableReader sstable : sstablesForShard) + { + int shardsSpanned = ArenaSelector.shardsSpanned(sstable, boundaries); + // calculating manually instead of calling ArenaSelector.shardAdjustedSize to save 1 call to ArenaSelector.shardsSpanned + shardAdjustedSize += sstable.onDiskLength() / shardsSpanned; + shardAdjustedKeyCount += sstable.estimatedKeys() / shardsSpanned; + } + + if (shardAdjustedSize > minSstableSizeInBytes) + break; + } + + return Math.round(shardAdjustedKeyCount * overwriteRatio); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/unified/ShardedMultiWriter.java b/src/java/org/apache/cassandra/db/compaction/unified/ShardedMultiWriter.java new file mode 100644 index 000000000000..fe129b9cf0b2 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/ShardedMultiWriter.java @@ -0,0 +1,272 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.UUID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.utils.FBUtilities; + +/** + * A {@link SSTableMultiWriter} that splits the output sstable at the partition boundaries of the compaction + * shards used by {@link org.apache.cassandra.db.compaction.UnifiedCompactionStrategy} as long as the size of + * the sstable so far is sufficiently large. + *

    + * This is class is similar to {@link ShardedMultiWriter} but for flushing. Unfortunately + * we currently have 2 separate writers hierarchy that are not compatible and so we must + * duplicate the functionality of splitting sstables over compaction shards if they have + * reached a minimum size. + */ +public class ShardedMultiWriter implements SSTableMultiWriter +{ + protected final static Logger logger = LoggerFactory.getLogger(ShardedMultiWriter.class); + + private final ColumnFamilyStore cfs; + private final Descriptor descriptor; + private final long keyCount; + private final long repairedAt; + private final UUID pendingRepair; + private final boolean isTransient; + private final MetadataCollector meta; + private final SerializationHeader header; + private final Collection indexGroups; + private final LifecycleNewTracker lifecycleNewTracker; + private final long minSstableSizeInBytes; + private final List boundaries; + private final SSTableMultiWriter[] writers; + private final int estimatedSSTables; + private int currentBoundary; + private int currentWriter; + + public ShardedMultiWriter(ColumnFamilyStore cfs, + Descriptor descriptor, + long keyCount, + long repairedAt, + UUID pendingRepair, + boolean isTransient, + MetadataCollector meta, + SerializationHeader header, + Collection indexGroups, + LifecycleNewTracker lifecycleNewTracker, + long minSstableSizeInBytes, + List boundaries) + { + this.cfs = cfs; + this.descriptor = descriptor; + this.keyCount = keyCount; + this.repairedAt = repairedAt; + this.pendingRepair = pendingRepair; + this.isTransient = isTransient; + this.meta = meta; + this.header = header; + this.indexGroups = indexGroups; + this.lifecycleNewTracker = lifecycleNewTracker; + this.minSstableSizeInBytes = minSstableSizeInBytes; + this.boundaries = boundaries; + this.writers = new SSTableMultiWriter[boundaries.size()]; + this.estimatedSSTables = (int) Math.max(1, Math.ceil(cfs.getFlushSizeOnDisk() / minSstableSizeInBytes)); + + this.currentBoundary = 0; + this.currentWriter = 0; + this.writers[currentWriter] = createWriter(descriptor); + } + + private SSTableMultiWriter createWriter() + { + Descriptor newDesc = cfs.newSSTableDescriptor(descriptor.directory); + return createWriter(newDesc); + } + + private SSTableMultiWriter createWriter(Descriptor desc) + { + return SimpleSSTableMultiWriter.create(desc, + forSplittingKeysBy(estimatedSSTables), + repairedAt, + pendingRepair, + isTransient, + cfs.metadata, + meta, + header, + indexGroups, + lifecycleNewTracker); + } + + private long forSplittingKeysBy(long splits) { + return splits <= 1 ? keyCount : keyCount / splits; + } + + @Override + public boolean append(UnfilteredRowIterator partition) + { + DecoratedKey key = partition.partitionKey(); + + boolean boundaryCrossed = false; + /* + The comparison to detect a boundary is costly, but if we only do this when the size is above the threshold, + we may detect a boundary change in the middle of a shard and split sstables at the wrong place. + */ + while (currentBoundary < boundaries.size() && key.compareTo(boundaries.get(currentBoundary)) >= 0) + { + currentBoundary++; + if (!boundaryCrossed) + boundaryCrossed = true; + } + + if (boundaryCrossed && writers[currentWriter].getOnDiskBytesWritten() >= minSstableSizeInBytes) + { + logger.debug("Switching writer at boundary {}/{} index {}/{}, with size {} for {}.{}", + key.getToken(), boundaries.get(currentBoundary-1), currentBoundary-1, currentWriter, + FBUtilities.prettyPrintMemory(writers[currentWriter].getBytesWritten()), + cfs.getKeyspaceName(), cfs.getTableName()); + + writers[++currentWriter] = createWriter(); + } + + return writers[currentWriter].append(partition); + } + + @Override + public Collection finish(long repairedAt, long maxDataAge, boolean openResult) + { + List sstables = new ArrayList<>(writers.length); + for (SSTableMultiWriter writer : writers) + if (writer != null) + sstables.addAll(writer.finish(repairedAt, maxDataAge, openResult)); + return sstables; + } + + @Override + public Collection finish(boolean openResult) + { + List sstables = new ArrayList<>(writers.length); + for (SSTableMultiWriter writer : writers) + if (writer != null) + sstables.addAll(writer.finish(openResult)); + return sstables; + } + + @Override + public Collection finished() + { + List sstables = new ArrayList<>(writers.length); + for (SSTableMultiWriter writer : writers) + if (writer != null) + sstables.addAll(writer.finished()); + return sstables; + } + + @Override + public SSTableMultiWriter setOpenResult(boolean openResult) + { + for (SSTableMultiWriter writer : writers) + if (writer != null) + writer.setOpenResult(openResult); + return this; + } + + @Override + public String getFilename() + { + for (SSTableMultiWriter writer : writers) + if (writer != null) + return writer.getFilename(); + return ""; + } + + @Override + public long getBytesWritten() + { + long bytesWritten = 0; + for (int i = 0; i <= currentWriter; ++i) + bytesWritten += writers[i].getBytesWritten(); + return bytesWritten; + } + + @Override + public long getOnDiskBytesWritten() + { + long bytesWritten = 0; + for (int i = 0; i <= currentWriter; ++i) + bytesWritten += writers[i].getOnDiskBytesWritten(); + return bytesWritten; + } + + public int getSegmentCount() + { + return currentWriter + 1; + } + + @Override + public TableId getTableId() + { + return cfs.metadata.id; + } + + @Override + public Throwable commit(Throwable accumulate) + { + Throwable t = accumulate; + for (SSTableMultiWriter writer : writers) + if (writer != null) + t = writer.commit(t); + return t; + } + + @Override + public Throwable abort(Throwable accumulate) + { + Throwable t = accumulate; + for (SSTableMultiWriter writer : writers) + if (writer != null) + t = writer.abort(t); + return t; + } + + @Override + public void prepareToCommit() + { + for (SSTableMultiWriter writer : writers) + if (writer != null) + writer.prepareToCommit(); + } + + @Override + public void close() + { + for (SSTableMultiWriter writer : writers) + if (writer != null) + writer.close(); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/unified/StaticController.java b/src/java/org/apache/cassandra/db/compaction/unified/StaticController.java new file mode 100644 index 000000000000..1d5578127927 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/StaticController.java @@ -0,0 +1,129 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.Arrays; +import java.util.Map; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.MonotonicClock; + +/** + * The static compaction controller periodically checks the IO costs + * that result from the current configuration of the {@link UnifiedCompactionStrategy}. + */ +public class StaticController extends Controller +{ + /** + * The scaling parameters W, one per bucket index and separated by a comma. + * Higher indexes will use the value of the last index with a W specified. + */ + final static String STATIC_SCALING_PARAMETERS_OPTION = "static_scaling_parameters"; + private final static String DEFAULT_STATIC_SCALING_PARAMETERS = System.getProperty(PREFIX + STATIC_SCALING_PARAMETERS_OPTION, "2"); + + private final int[] scalingParameters; + + @VisibleForTesting // comp. simulation + public StaticController(Environment env, + int[] scalingParameters, + double survivalFactor, + long dataSetSizeMB, + int numShards, + long minSSTableSizeMB, + long flushSizeOverrideMB, + double maxSpaceOverhead, + int maxSSTablesToCompact, + long expiredSSTableCheckFrequency, + boolean ignoreOverlapsInExpirationCheck) + { + super(MonotonicClock.preciseTime, + env, + survivalFactor, + dataSetSizeMB, + numShards, + minSSTableSizeMB, + flushSizeOverrideMB, + maxSpaceOverhead, + maxSSTablesToCompact, + expiredSSTableCheckFrequency, + ignoreOverlapsInExpirationCheck); + this.scalingParameters = scalingParameters; + } + + static Controller fromOptions(Environment env, + double survivalFactor, + long dataSetSizeMB, + int numShards, + long minSSTableSizeMB, + long flushSizeOverrideMB, + double maxSpaceOverhead, + int maxSSTablesToCompact, + long expiredSSTableCheckFrequency, + boolean ignoreOverlapsInExpirationCheck, + Map options) + { + int[] Ws = parseScalingParameters(options.getOrDefault(STATIC_SCALING_PARAMETERS_OPTION, DEFAULT_STATIC_SCALING_PARAMETERS)); + return new StaticController(env, + Ws, + survivalFactor, + dataSetSizeMB, + numShards, + minSSTableSizeMB, + flushSizeOverrideMB, + maxSpaceOverhead, + maxSSTablesToCompact, + expiredSSTableCheckFrequency, + ignoreOverlapsInExpirationCheck); + } + + @VisibleForTesting + static int[] parseScalingParameters(String str) + { + String[] vals = str.split(","); + int[] ret = new int[vals.length]; + for (int i = 0; i < vals.length; i++) + ret[i] = Integer.parseInt(vals[i].trim()); + + return ret; + } + + public static Map validateOptions(Map options) throws ConfigurationException + { + String s = options.remove(STATIC_SCALING_PARAMETERS_OPTION); + if (s != null) + parseScalingParameters(s); + return options; + } + + @Override + public int getScalingParameter(int index) + { + if (index < 0) + throw new IllegalArgumentException("Index should be >= 0: " + index); + + return index < scalingParameters.length ? scalingParameters[index] : scalingParameters[scalingParameters.length - 1]; + } + + @Override + public String toString() + { + return String.format("Static controller, m: %d, o: %f, Ws: %s, cost: %s", minSstableSizeMB, survivalFactor, Arrays.toString(scalingParameters), calculator); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/unified/UnifiedCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/unified/UnifiedCompactionTask.java new file mode 100644 index 000000000000..7df03fe970f2 --- /dev/null +++ b/src/java/org/apache/cassandra/db/compaction/unified/UnifiedCompactionTask.java @@ -0,0 +1,59 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed 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.compaction.unified; + +import java.util.List; +import java.util.Set; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.compaction.CompactionTask; +import org.apache.cassandra.db.compaction.UnifiedCompactionStrategy; +import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.io.sstable.format.SSTableReader; + +/** + * The sole purpose of this class is to currently create a {@link ShardedCompactionWriter}. + */ +public class UnifiedCompactionTask extends CompactionTask +{ + private final long minSstableSizeInBytes; + private final List boundaries; + + public UnifiedCompactionTask(ColumnFamilyStore cfs, + UnifiedCompactionStrategy strategy, + LifecycleTransaction txn, + int gcBefore, + long minSstableSizeInBytes, + List boundaries) + { + super(cfs, txn, gcBefore, strategy.getController().getIgnoreOverlapsInExpirationCheck(), strategy); + this.minSstableSizeInBytes = minSstableSizeInBytes; + this.boundaries = boundaries; + } + + @Override + public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, + Directories directories, + LifecycleTransaction txn, + Set nonExpiredSSTables) + { + return new ShardedCompactionWriter(cfs, directories, txn, nonExpiredSSTables, keepOriginals, minSstableSizeInBytes, boundaries); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java index d363dcf3daf3..61e97ac09717 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java @@ -18,7 +18,6 @@ package org.apache.cassandra.db.compaction.writers; -import java.io.File; import java.util.Collection; import java.util.List; import java.util.Set; @@ -32,15 +31,18 @@ import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.DiskBoundaries; import org.apache.cassandra.db.PartitionPosition; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.compaction.CompactionTask; +import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SSTableRewriter; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.Transactional; -import org.apache.cassandra.db.compaction.OperationType; /** @@ -65,6 +67,7 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa private final List locations; private final List diskBoundaries; private int locationIndex; + protected Directories.DataDirectory currentDirectory; public CompactionAwareWriter(ColumnFamilyStore cfs, Directories directories, @@ -84,7 +87,7 @@ public CompactionAwareWriter(ColumnFamilyStore cfs, pendingRepair = CompactionTask.getPendingRepair(nonExpiredSSTables); isTransient = CompactionTask.getIsTransient(nonExpiredSSTables); DiskBoundaries db = cfs.getDiskBoundaries(); - diskBoundaries = db.positions; + diskBoundaries = db.getPositions(); locations = db.directories; locationIndex = -1; } @@ -131,10 +134,10 @@ public long estimatedKeys() * @param partition the partition to append * @return true if the partition was written, false otherwise */ - public final boolean append(UnfilteredRowIterator partition) + public boolean append(UnfilteredRowIterator partition) { maybeSwitchWriter(partition.partitionKey()); - return realAppend(partition); + return sstableWriter.append(partition) != null; } @Override @@ -144,43 +147,87 @@ protected Throwable doPostCleanup(Throwable accumulate) return super.doPostCleanup(accumulate); } - protected abstract boolean realAppend(UnfilteredRowIterator partition); - /** + * Switches the writer if necessary, i.e. if the new key should be placed in a different data directory, or if the + * specific strategy has decided a new sstable is needed. + * * Guaranteed to be called before the first call to realAppend. - * @param key */ protected void maybeSwitchWriter(DecoratedKey key) + { + if (maybeSwitchLocation(key)) + return; + + if (shouldSwitchWriterInCurrentLocation(key)) + switchCompactionWriter(currentDirectory); + } + + /** + * Switches the file location and writer and returns true if the new key should be placed in a different data + * directory. + */ + protected boolean maybeSwitchLocation(DecoratedKey key) { if (diskBoundaries == null) { if (locationIndex < 0) { Directories.DataDirectory defaultLocation = getWriteDirectory(nonExpiredSSTables, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, OperationType.UNKNOWN)); - switchCompactionLocation(defaultLocation); + switchCompactionWriter(defaultLocation); locationIndex = 0; + return true; } - return; + return false; } if (locationIndex > -1 && key.compareTo(diskBoundaries.get(locationIndex)) < 0) - return; + return false; int prevIdx = locationIndex; while (locationIndex == -1 || key.compareTo(diskBoundaries.get(locationIndex)) > 0) locationIndex++; + Directories.DataDirectory newLocation = locations.get(locationIndex); if (prevIdx >= 0) - logger.debug("Switching write location from {} to {}", locations.get(prevIdx), locations.get(locationIndex)); - switchCompactionLocation(locations.get(locationIndex)); + logger.debug("Switching write location from {} to {}", locations.get(prevIdx), newLocation); + switchCompactionWriter(newLocation); + return true; } + /** + * Returns true if the writer should be switched for reasons other than switching to a new data directory + * (e.g. because an sstable size limit has been reached). + */ + protected abstract boolean shouldSwitchWriterInCurrentLocation(DecoratedKey key); + /** * Implementations of this method should finish the current sstable writer and start writing to this directory. * * Called once before starting to append and then whenever we see a need to start writing to another directory. * @param directory */ - protected abstract void switchCompactionLocation(Directories.DataDirectory directory); + protected void switchCompactionWriter(Directories.DataDirectory directory) + { + currentDirectory = directory; + PartitionPosition diskBoundary = diskBoundaries != null && locationIndex > -1 + ? diskBoundaries.get(locationIndex) + : null; + sstableWriter.switchWriter(sstableWriter(directory, diskBoundary)); + } + + @SuppressWarnings("resource") + protected SSTableWriter sstableWriter(Directories.DataDirectory directory, PartitionPosition diskBoundary) + { + return SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(directory)), + estimatedTotalKeys, + minRepairedAt, + pendingRepair, + isTransient, + cfs.metadata, + new MetadataCollector(txn.originals(), cfs.metadata().comparator), + SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), + cfs.indexManager.listIndexGroups(), + txn); + } /** * The directories we can write to @@ -233,4 +280,9 @@ public CompactionAwareWriter setRepairedAt(long repairedAt) this.sstableWriter.setRepairedAt(repairedAt); return this; } + + public long bytesWritten() + { + return sstableWriter.bytesWritten(); + } } diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java index 6180f96100b5..d51184f336b6 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java @@ -24,9 +24,10 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.SerializationHeader; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableWriter; @@ -59,26 +60,25 @@ public DefaultCompactionWriter(ColumnFamilyStore cfs, Directories directories, L } @Override - public boolean realAppend(UnfilteredRowIterator partition) + protected boolean shouldSwitchWriterInCurrentLocation(DecoratedKey key) { - return sstableWriter.append(partition) != null; + return false; } + @SuppressWarnings("resource") @Override - public void switchCompactionLocation(Directories.DataDirectory directory) + protected SSTableWriter sstableWriter(Directories.DataDirectory directory, PartitionPosition diskBoundary) { - @SuppressWarnings("resource") - SSTableWriter writer = SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(directory)), - estimatedTotalKeys, - minRepairedAt, - pendingRepair, - isTransient, - cfs.metadata, - new MetadataCollector(txn.originals(), cfs.metadata().comparator, sstableLevel), - SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), - cfs.indexManager.listIndexes(), - txn); - sstableWriter.switchWriter(writer); + return SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(directory)), + estimatedTotalKeys, + minRepairedAt, + pendingRepair, + isTransient, + cfs.metadata, + new MetadataCollector(txn.originals(), cfs.metadata().comparator, sstableLevel), + SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), + cfs.indexManager.listIndexGroups(), + txn); } @Override diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java index 1c5360020b49..240ccc504a1c 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java @@ -20,13 +20,14 @@ import java.util.Set; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.RowIndexEntry; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.compaction.LeveledManifest; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; -import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableWriter; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; @@ -39,7 +40,6 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter private long totalWrittenInLevel = 0; private int sstablesWritten = 0; private final long keysPerSSTable; - private Directories.DataDirectory sstableDirectory; private final int levelFanoutSize; public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, @@ -67,11 +67,15 @@ public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, } @Override - @SuppressWarnings("resource") - public boolean realAppend(UnfilteredRowIterator partition) + public boolean append(UnfilteredRowIterator partition) { - RowIndexEntry rie = sstableWriter.append(partition); partitionsWritten++; + return super.append(partition); + } + + @Override + protected boolean shouldSwitchWriterInCurrentLocation(DecoratedKey key) + { long totalWrittenInCurrentWriter = sstableWriter.currentWriter().getEstimatedOnDiskBytesWritten(); if (totalWrittenInCurrentWriter > maxSSTableSize) { @@ -81,28 +85,34 @@ public boolean realAppend(UnfilteredRowIterator partition) totalWrittenInLevel = 0; currentLevel++; } - switchCompactionLocation(sstableDirectory); + return true; } - return rie != null; + return false; } @Override - public void switchCompactionLocation(Directories.DataDirectory location) + public void switchCompactionWriter(Directories.DataDirectory location) { - this.sstableDirectory = location; averageEstimatedKeysPerSSTable = Math.round(((double) averageEstimatedKeysPerSSTable * sstablesWritten + partitionsWritten) / (sstablesWritten + 1)); - sstableWriter.switchWriter(SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(sstableDirectory)), - keysPerSSTable, - minRepairedAt, - pendingRepair, - isTransient, - cfs.metadata, - new MetadataCollector(txn.originals(), cfs.metadata().comparator, currentLevel), - SerializationHeader.make(cfs.metadata(), txn.originals()), - cfs.indexManager.listIndexes(), - txn)); partitionsWritten = 0; sstablesWritten = 0; + super.switchCompactionWriter(location); + } + + @Override + @SuppressWarnings("resource") + protected SSTableWriter sstableWriter(Directories.DataDirectory directory, PartitionPosition diskBoundary) + { + return SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(directory)), + keysPerSSTable, + minRepairedAt, + pendingRepair, + isTransient, + cfs.metadata, + new MetadataCollector(txn.originals(), cfs.metadata().comparator, currentLevel), + SerializationHeader.make(cfs.metadata(), txn.originals()), + cfs.indexManager.listIndexGroups(), + txn); } } diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java index 915f96bfb431..a094d4253102 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java @@ -20,11 +20,11 @@ import java.util.Set; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.RowIndexEntry; +import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.compaction.OperationType; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableWriter; @@ -35,8 +35,6 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter private final long maxSSTableSize; private final int level; private final long estimatedSSTables; - private final Set allSSTables; - private Directories.DataDirectory sstableDirectory; public MaxSSTableSizeWriter(ColumnFamilyStore cfs, Directories directories, @@ -57,7 +55,6 @@ public MaxSSTableSizeWriter(ColumnFamilyStore cfs, boolean keepOriginals) { super(cfs, directories, txn, nonExpiredSSTables, keepOriginals); - this.allSSTables = txn.originals(); this.level = level; this.maxSSTableSize = maxSSTableSize; @@ -79,32 +76,24 @@ private static long getTotalWriteSize(Iterable nonExpiredSSTables return Math.round(estimatedCompactionRatio * cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType)); } - protected boolean realAppend(UnfilteredRowIterator partition) + @Override + protected boolean shouldSwitchWriterInCurrentLocation(DecoratedKey key) { - RowIndexEntry rie = sstableWriter.append(partition); - if (sstableWriter.currentWriter().getEstimatedOnDiskBytesWritten() > maxSSTableSize) - { - switchCompactionLocation(sstableDirectory); - } - return rie != null; + return sstableWriter.currentWriter().getEstimatedOnDiskBytesWritten() > maxSSTableSize; } @Override - public void switchCompactionLocation(Directories.DataDirectory location) + protected SSTableWriter sstableWriter(Directories.DataDirectory directory, PartitionPosition diskBoundary) { - sstableDirectory = location; - @SuppressWarnings("resource") - SSTableWriter writer = SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(sstableDirectory)), - estimatedTotalKeys / estimatedSSTables, - minRepairedAt, - pendingRepair, - isTransient, - cfs.metadata, - new MetadataCollector(allSSTables, cfs.metadata().comparator, level), - SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), - cfs.indexManager.listIndexes(), - txn); - - sstableWriter.switchWriter(writer); + return SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(directory)), + estimatedTotalKeys / estimatedSSTables, + minRepairedAt, + pendingRepair, + isTransient, + cfs.metadata, + new MetadataCollector(txn.originals(), cfs.metadata().comparator, level), + SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), + cfs.indexManager.listIndexGroups(), + txn); } } diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java index d29061ca8630..c389d81186f7 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java @@ -24,10 +24,10 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.RowIndexEntry; +import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.SerializationHeader; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableWriter; @@ -49,7 +49,6 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter private final Set allSSTables; private long currentBytesToWrite; private int currentRatioIndex = 0; - private Directories.DataDirectory location; public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Directories directories, LifecycleTransaction txn, Set nonExpiredSSTables) { @@ -84,36 +83,33 @@ public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Directories di } @Override - public boolean realAppend(UnfilteredRowIterator partition) + protected boolean shouldSwitchWriterInCurrentLocation(DecoratedKey key) { - RowIndexEntry rie = sstableWriter.append(partition); if (sstableWriter.currentWriter().getEstimatedOnDiskBytesWritten() > currentBytesToWrite && currentRatioIndex < ratios.length - 1) // if we underestimate how many keys we have, the last sstable might get more than we expect { currentRatioIndex++; currentBytesToWrite = Math.round(totalSize * ratios[currentRatioIndex]); - switchCompactionLocation(location); logger.debug("Switching writer, currentBytesToWrite = {}", currentBytesToWrite); + return true; } - return rie != null; + return false; } @Override - public void switchCompactionLocation(Directories.DataDirectory location) + protected SSTableWriter sstableWriter(Directories.DataDirectory directory, PartitionPosition diskBoundary) { - this.location = location; long currentPartitionsToWrite = Math.round(ratios[currentRatioIndex] * estimatedTotalKeys); - @SuppressWarnings("resource") - SSTableWriter writer = SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(location)), - currentPartitionsToWrite, - minRepairedAt, - pendingRepair, - isTransient, - cfs.metadata, - new MetadataCollector(allSSTables, cfs.metadata().comparator, 0), - SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), - cfs.indexManager.listIndexes(), - txn); logger.trace("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite); - sstableWriter.switchWriter(writer); + + return SSTableWriter.create(cfs.newSSTableDescriptor(getDirectories().getLocationForDisk(directory)), + currentPartitionsToWrite, + minRepairedAt, + pendingRepair, + isTransient, + cfs.metadata, + new MetadataCollector(allSSTables, cfs.metadata().comparator, 0), + SerializationHeader.make(cfs.metadata(), nonExpiredSSTables), + cfs.indexManager.listIndexGroups(), + txn); } } diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java index 6ea0435b54c3..8d3b350988fd 100644 --- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java @@ -23,7 +23,6 @@ import org.apache.cassandra.db.partitions.CachedPartition; import org.apache.cassandra.db.partitions.Partition; import org.apache.cassandra.db.rows.*; -import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.TableMetadata; @@ -142,13 +141,14 @@ static interface InternalDeserializer public UnfilteredRowIterator getUnfilteredRowIterator(ColumnFilter columnFilter, Partition partition); /** - * Whether the provided sstable may contain data that is selected by this filter (based on the sstable metadata). + * Whether the data selected by this filter intersects with the provided slice. * - * @param sstable the sstable for which we want to test the need for inclusion. + * @param comparator the comparator of the table this if a filter on. + * @param slice the slice to check intersection with, * - * @return whether {@code sstable} should be included to answer this filter. + * @return whether the data selected by this filter intersects with {@code slice}. */ - public boolean shouldInclude(SSTableReader sstable); + public boolean intersects(ClusteringComparator comparator, Slice slice); public Kind kind(); diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java index ef9ceff9ef1e..f61a39aa87d4 100644 --- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java @@ -18,14 +18,12 @@ package org.apache.cassandra.db.filter; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.*; import org.apache.cassandra.db.*; import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.db.transform.Transformation; -import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; @@ -140,16 +138,11 @@ public UnfilteredRowIterator getUnfilteredRowIterator(final ColumnFilter columnF return partition.unfilteredIterator(columnFilter, clusteringsInQueryOrder, isReversed()); } - public boolean shouldInclude(SSTableReader sstable) + public boolean intersects(ClusteringComparator comparator, Slice slice) { - ClusteringComparator comparator = sstable.metadata().comparator; - List minClusteringValues = sstable.getSSTableMetadata().minClusteringValues; - List maxClusteringValues = sstable.getSSTableMetadata().maxClusteringValues; - - // If any of the requested clustering is within the bounds covered by the sstable, we need to include the sstable for (Clustering clustering : clusterings) { - if (Slice.make(clustering).intersects(comparator, minClusteringValues, maxClusteringValues)) + if (slice.includes(comparator, clustering)) return true; } return false; diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java index 5df98c3e70c4..5b72a6f91bbe 100644 --- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java @@ -18,8 +18,6 @@ package org.apache.cassandra.db.filter; import java.io.IOException; -import java.util.List; -import java.nio.ByteBuffer; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.*; @@ -27,7 +25,6 @@ import org.apache.cassandra.db.partitions.CachedPartition; import org.apache.cassandra.db.partitions.Partition; import org.apache.cassandra.db.transform.Transformation; -import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; @@ -126,15 +123,9 @@ public UnfilteredRowIterator getUnfilteredRowIterator(ColumnFilter columnFilter, return partition.unfilteredIterator(columnFilter, slices, reversed); } - public boolean shouldInclude(SSTableReader sstable) + public boolean intersects(ClusteringComparator comparator, Slice slice) { - List minClusteringValues = sstable.getSSTableMetadata().minClusteringValues; - List maxClusteringValues = sstable.getSSTableMetadata().maxClusteringValues; - - if (minClusteringValues.isEmpty() || maxClusteringValues.isEmpty()) - return true; - - return slices.intersects(minClusteringValues, maxClusteringValues); + return slices.intersects(slice); } public String toString(TableMetadata metadata) diff --git a/src/java/org/apache/cassandra/db/filter/DataLimits.java b/src/java/org/apache/cassandra/db/filter/DataLimits.java index 845cffd4457d..a32eda5e180a 100644 --- a/src/java/org/apache/cassandra/db/filter/DataLimits.java +++ b/src/java/org/apache/cassandra/db/filter/DataLimits.java @@ -19,34 +19,64 @@ import java.io.IOException; import java.nio.ByteBuffer; - -import org.apache.cassandra.db.*; +import java.util.ArrayList; +import java.util.List; +import java.util.StringJoiner; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cql3.PageSize; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.aggregation.AggregationSpecification; import org.apache.cassandra.db.aggregation.GroupMaker; import org.apache.cassandra.db.aggregation.GroupingState; -import org.apache.cassandra.db.aggregation.AggregationSpecification; -import org.apache.cassandra.db.rows.*; -import org.apache.cassandra.db.partitions.*; +import org.apache.cassandra.db.partitions.CachedPartition; +import org.apache.cassandra.db.partitions.PartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.BaseRowIterator; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.RowIterator; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.transform.BasePartitions; import org.apache.cassandra.db.transform.BaseRows; import org.apache.cassandra.db.transform.StoppingTransformation; import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.utils.ByteBufferUtil; /** - * Object in charge of tracking if we have fetch enough data for a given query. - * - * This is more complicated than a single count because we support PER PARTITION - * limits, but also due to GROUP BY and paging. + * Object in charge of tracking if we have fetched enough data for a given query. + *

    + * This is more complicated than a single count because we support {@code PER PARTITION} + * limits, but also due to {@code GROUP BY} and paging. + *

    + *

    + * Tracking happens by row count ({@see count()}) and bytes ({@see bytes()}), with the first exhausted limit + * taking precedence. + *

    + *

    + * When paging is used (see {@code forPaging} methods), the minimum number between the page size and the rows/bytes + * limit is enforced, meaning that we'll never return more rows than requested. + *

    */ public abstract class DataLimits { + private static final Logger logger = LoggerFactory.getLogger(DataLimits.class); public static final Serializer serializer = new Serializer(); public static final int NO_LIMIT = Integer.MAX_VALUE; - public static final DataLimits NONE = new CQLLimits(NO_LIMIT) + public static final DataLimits NONE = new CQLLimits(NO_LIMIT, NO_LIMIT, NO_LIMIT, false) { @Override public boolean hasEnoughLiveData(CachedPartition cached, int nowInSec, boolean countPartitionsWithOnlyStaticData, boolean enforceStrictLiveness) @@ -79,7 +109,7 @@ public PartitionIterator filter(PartitionIterator iter, int nowInSec, boolean co // We currently deal with distinct queries by querying full partitions but limiting the result at 1 row per // partition (see SelectStatement.makeFilter). So an "unbounded" distinct is still actually doing some filtering. - public static final DataLimits DISTINCT_NONE = new CQLLimits(NO_LIMIT, 1, true); + public static final DataLimits DISTINCT_NONE = new CQLLimits(NO_LIMIT, NO_LIMIT, 1, true); public enum Kind { @@ -93,29 +123,30 @@ public enum Kind public static DataLimits cqlLimits(int cqlRowLimit) { - return cqlRowLimit == NO_LIMIT ? NONE : new CQLLimits(cqlRowLimit); + return cqlRowLimit == NO_LIMIT ? NONE : new CQLLimits(NO_LIMIT, cqlRowLimit, NO_LIMIT, false); } public static DataLimits cqlLimits(int cqlRowLimit, int perPartitionLimit) { return cqlRowLimit == NO_LIMIT && perPartitionLimit == NO_LIMIT ? NONE - : new CQLLimits(cqlRowLimit, perPartitionLimit); + : new CQLLimits(NO_LIMIT, cqlRowLimit, perPartitionLimit, false); } - private static DataLimits cqlLimits(int cqlRowLimit, int perPartitionLimit, boolean isDistinct) + private static DataLimits cqlLimits(int bytesLimit, int cqlRowLimit, int perPartitionLimit, boolean isDistinct) { - return cqlRowLimit == NO_LIMIT && perPartitionLimit == NO_LIMIT && !isDistinct + return bytesLimit == NO_LIMIT && cqlRowLimit == NO_LIMIT && perPartitionLimit == NO_LIMIT && !isDistinct ? NONE - : new CQLLimits(cqlRowLimit, perPartitionLimit, isDistinct); + : new CQLLimits(bytesLimit, cqlRowLimit, perPartitionLimit, isDistinct); } public static DataLimits groupByLimits(int groupLimit, int groupPerPartitionLimit, + int bytesLimit, int rowLimit, AggregationSpecification groupBySpec) { - return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, rowLimit, groupBySpec); + return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec); } public static DataLimits distinctLimits(int cqlRowLimit) @@ -133,13 +164,20 @@ public boolean isGroupByLimit() return false; } - public boolean isExhausted(Counter counter) + /** + * Returns true if the count limit is not reached. + * + * Note: currently this method's only usage is for paging, where it is checked after processing a page as a quick + * signal that the data for the query is complete - if the count limit is not reached at the end of the page, this + * must be because there is no more data to return. + */ + public boolean isCounterBelowLimits(Counter counter) { - return counter.counted() < count(); + return counter.counted() < count() && counter.bytesCounted() < bytes(); } - public abstract DataLimits forPaging(int pageSize); - public abstract DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining); + public abstract DataLimits forPaging(PageSize pageSize); + public abstract DataLimits forPaging(PageSize pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining); public abstract DataLimits forShortReadRetry(int toFetch); @@ -177,6 +215,23 @@ public abstract Counter newCounter(int nowInSec, boolean countPartitionsWithOnlyStaticData, boolean enforceStrictLiveness); + /** + * The max number of bytes this limits enforces. + *

    + * Note that if this value is set, less rows might be returned if the size of the current rows exceeds the bytes limit. + * + * @return the maximum number of bytes this limits enforces. + */ + public abstract int bytes(); + + /** + * The max number of rows this limits enforces. Note that this means traversed rows, regardless we use grouping or not. + *

    + * @return the maximum number of rows this limits enforces. + */ + @VisibleForTesting + public abstract int rows(); + /** * The max number of results this limits enforces. *

    @@ -195,6 +250,17 @@ public abstract Counter newCounter(int nowInSec, */ public abstract DataLimits withoutState(); + /** + * Returns a copy of this DataLimits with updated counted limit whatever it is (either the rows limit + * or groups limit depending on the actual implementation) + */ + public abstract DataLimits withCountedLimit(int newCountedLimit); + + /** + * Returns a copy of this DataLimits with updated bytes limit. + */ + public abstract DataLimits withBytesLimit(int bytesLimit); + public UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int nowInSec, boolean countPartitionsWithOnlyStaticData) @@ -280,6 +346,12 @@ public RowIterator applyTo(RowIterator partition) public abstract int countedInCurrentPartition(); + /** + * The number of bytes for the counted rows. + * + * @return the number of bytes counted. + */ + public abstract int bytesCounted(); /** * The number of rows counted. * @@ -339,36 +411,32 @@ public void onClose() } /** - * Limits used by CQL; this counts rows. + * Limits used by CQL; this counts rows or bytes read. Please note: + *

      + *
    • When paging on rows, the minimum number of rows between the current limit and the page size is used as actual limit.
    • + *
    • When paging on bytes, the number of bytes takes precedence over the rows limit.
    • + *
    */ private static class CQLLimits extends DataLimits { + protected final int bytesLimit; protected final int rowLimit; protected final int perPartitionLimit; // Whether the query is a distinct query or not. protected final boolean isDistinct; - private CQLLimits(int rowLimit) - { - this(rowLimit, NO_LIMIT); - } - - private CQLLimits(int rowLimit, int perPartitionLimit) - { - this(rowLimit, perPartitionLimit, false); - } - - private CQLLimits(int rowLimit, int perPartitionLimit, boolean isDistinct) + private CQLLimits(int bytesLimit, int rowsLimit, int perPartitionLimit, boolean isDistinct) { - this.rowLimit = rowLimit; + this.bytesLimit = bytesLimit; + this.rowLimit = rowsLimit; this.perPartitionLimit = perPartitionLimit; this.isDistinct = isDistinct; } private static CQLLimits distinct(int rowLimit) { - return new CQLLimits(rowLimit, 1, true); + return new CQLLimits(NO_LIMIT, rowLimit, 1, true); } public Kind kind() @@ -378,7 +446,7 @@ public Kind kind() public boolean isUnlimited() { - return rowLimit == NO_LIMIT && perPartitionLimit == NO_LIMIT; + return bytesLimit == NO_LIMIT && rowLimit == NO_LIMIT && perPartitionLimit == NO_LIMIT; } public boolean isDistinct() @@ -386,19 +454,27 @@ public boolean isDistinct() return isDistinct; } - public DataLimits forPaging(int pageSize) + public DataLimits forPaging(PageSize pageSize) { - return new CQLLimits(pageSize, perPartitionLimit, isDistinct); + return new CQLLimits(pageSize.minBytesCount(bytesLimit), + pageSize.minRowsCount(rowLimit), + perPartitionLimit, + isDistinct); } - public DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) + public DataLimits forPaging(PageSize pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) { - return new CQLPagingLimits(pageSize, perPartitionLimit, isDistinct, lastReturnedKey, lastReturnedKeyRemaining); + return new CQLPagingLimits(pageSize.minBytesCount(bytesLimit), + pageSize.minRowsCount(rowLimit), + perPartitionLimit, + isDistinct, + lastReturnedKey, + lastReturnedKeyRemaining); } public DataLimits forShortReadRetry(int toFetch) { - return new CQLLimits(toFetch, perPartitionLimit, isDistinct); + return new CQLLimits(bytesLimit, toFetch, perPartitionLimit, isDistinct); } public boolean hasEnoughLiveData(CachedPartition cached, int nowInSec, boolean countPartitionsWithOnlyStaticData, boolean enforceStrictLiveness) @@ -435,6 +511,16 @@ public Counter newCounter(int nowInSec, return new CQLCounter(nowInSec, assumeLiveData, countPartitionsWithOnlyStaticData, enforceStrictLiveness); } + public int bytes() + { + return bytesLimit; + } + + public int rows() + { + return rowLimit; + } + public int count() { return rowLimit; @@ -450,6 +536,18 @@ public DataLimits withoutState() return this; } + @Override + public DataLimits withCountedLimit(int newCountedLimit) + { + return new CQLLimits(bytesLimit, newCountedLimit, perPartitionLimit, isDistinct); + } + + @Override + public DataLimits withBytesLimit(int bytesLimit) + { + return new CQLLimits(bytesLimit, rowLimit, perPartitionLimit, isDistinct); + } + public float estimateTotalResults(ColumnFamilyStore cfs) { // TODO: we should start storing stats on the number of rows (instead of the number of cells, which @@ -460,10 +558,16 @@ public float estimateTotalResults(ColumnFamilyStore cfs) protected class CQLCounter extends Counter { + /** + * Bytes and rows counted by this counter. + */ + protected int bytesCounted; protected int rowsCounted; protected int rowsInCurrentPartition; protected final boolean countPartitionsWithOnlyStaticData; + protected int staticRowBytes; + protected boolean hasLiveStaticRow; public CQLCounter(int nowInSec, @@ -480,13 +584,14 @@ public void applyToPartition(DecoratedKey partitionKey, Row staticRow) { rowsInCurrentPartition = 0; hasLiveStaticRow = !staticRow.isEmpty() && isLive(staticRow); + staticRowBytes = hasLiveStaticRow && bytesLimit != NO_LIMIT ? staticRow.dataSize() : 0; } @Override public Row applyToRow(Row row) { if (isLive(row)) - incrementRowCount(); + incrementRowCount(bytesLimit != NO_LIMIT ? row.dataSize() : 0); return row; } @@ -497,15 +602,18 @@ public void onPartitionClose() // rows in the partition. However, if we only have the static row, it will be returned as one row // so count it. if (countPartitionsWithOnlyStaticData && hasLiveStaticRow && rowsInCurrentPartition == 0) - incrementRowCount(); + incrementRowCount(staticRowBytes); super.onPartitionClose(); } - protected void incrementRowCount() + protected void incrementRowCount(int rowSize) { - if (++rowsCounted >= rowLimit) + bytesCounted += rowSize; + rowsCounted++; + rowsInCurrentPartition++; + if (bytesCounted >= bytesLimit || rowsCounted >= rowLimit) stop(); - if (++rowsInCurrentPartition >= perPartitionLimit) + if (rowsInCurrentPartition >= perPartitionLimit) stopInPartition(); } @@ -519,6 +627,11 @@ public int countedInCurrentPartition() return rowsInCurrentPartition; } + public int bytesCounted() + { + return bytesCounted; + } + public int rowsCounted() { return rowsCounted; @@ -531,31 +644,35 @@ public int rowsCountedInCurrentPartition() public boolean isDone() { - return rowsCounted >= rowLimit; + return rowsCounted >= rowLimit || bytesCounted >= bytesLimit || counted() >= count(); } public boolean isDoneForPartition() { return isDone() || rowsInCurrentPartition >= perPartitionLimit; } + + @Override + public String toString() + { + return String.format("%s(bytes=%s/%s, rows=%s/%s, partition-rows=%s/%s)", this.getClass().getName(), + bytesCounted(), bytesLimit, rowsCounted(), rowLimit, rowsCountedInCurrentPartition(), perPartitionLimit); + } } @Override public String toString() { - StringBuilder sb = new StringBuilder(); + List limits = new ArrayList<>(3); + if (bytesLimit != NO_LIMIT) + limits.add("BYTES LIMIT " + bytesLimit); if (rowLimit != NO_LIMIT) - { - sb.append("LIMIT ").append(rowLimit); - if (perPartitionLimit != NO_LIMIT) - sb.append(' '); - } - + limits.add("ROWS LIMIT " + rowLimit); if (perPartitionLimit != NO_LIMIT) - sb.append("PER PARTITION LIMIT ").append(perPartitionLimit); + limits.add("PER PARTITION LIMIT " + perPartitionLimit); - return sb.toString(); + return String.join(" ", limits); } } @@ -564,9 +681,9 @@ private static class CQLPagingLimits extends CQLLimits private final ByteBuffer lastReturnedKey; private final int lastReturnedKeyRemaining; - public CQLPagingLimits(int rowLimit, int perPartitionLimit, boolean isDistinct, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) + public CQLPagingLimits(int bytesLimit, int rowLimit, int perPartitionLimit, boolean isDistinct, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) { - super(rowLimit, perPartitionLimit, isDistinct); + super(bytesLimit, rowLimit, perPartitionLimit, isDistinct); this.lastReturnedKey = lastReturnedKey; this.lastReturnedKeyRemaining = lastReturnedKeyRemaining; } @@ -578,13 +695,13 @@ public Kind kind() } @Override - public DataLimits forPaging(int pageSize) + public DataLimits forPaging(PageSize pageSize) { throw new UnsupportedOperationException(); } @Override - public DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) + public DataLimits forPaging(PageSize pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) { throw new UnsupportedOperationException(); } @@ -592,7 +709,19 @@ public DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastRe @Override public DataLimits withoutState() { - return new CQLLimits(rowLimit, perPartitionLimit, isDistinct); + return new CQLLimits(bytesLimit, rowLimit, perPartitionLimit, isDistinct); + } + + @Override + public DataLimits withCountedLimit(int newCountedLimit) + { + return new CQLPagingLimits(bytesLimit, newCountedLimit, perPartitionLimit, isDistinct, lastReturnedKey, lastReturnedKeyRemaining); + } + + @Override + public DataLimits withBytesLimit(int bytesLimit) + { + return new CQLPagingLimits(bytesLimit, rowLimit, perPartitionLimit, isDistinct, lastReturnedKey, lastReturnedKeyRemaining); } @Override @@ -622,6 +751,7 @@ public void applyToPartition(DecoratedKey partitionKey, Row staticRow) // if any already, so force hasLiveStaticRow to false so we make sure to not count it // once more. hasLiveStaticRow = false; + staticRowBytes = 0; } else { @@ -629,6 +759,16 @@ public void applyToPartition(DecoratedKey partitionKey, Row staticRow) } } } + + @Override + public String toString() + { + return new StringJoiner(", ", CQLPagingLimits.class.getSimpleName() + "[", "]") + .add("super=" + super.toString()) + .add("lastReturnedKey=" + (lastReturnedKey != null ? ByteBufferUtil.bytesToHex(lastReturnedKey) : null)) + .add("lastReturnedKeyRemaining=" + lastReturnedKeyRemaining) + .toString(); + } } /** @@ -661,19 +801,21 @@ private static class CQLGroupByLimits extends CQLLimits public CQLGroupByLimits(int groupLimit, int groupPerPartitionLimit, + int bytesLimit, int rowLimit, AggregationSpecification groupBySpec) { - this(groupLimit, groupPerPartitionLimit, rowLimit, groupBySpec, GroupingState.EMPTY_STATE); + this(groupLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec, GroupingState.EMPTY_STATE); } private CQLGroupByLimits(int groupLimit, int groupPerPartitionLimit, + int bytesLimit, int rowLimit, AggregationSpecification groupBySpec, GroupingState state) { - super(rowLimit, NO_LIMIT, false); + super(bytesLimit, rowLimit, NO_LIMIT, false); this.groupLimit = groupLimit; this.groupPerPartitionLimit = groupPerPartitionLimit; this.groupBySpec = groupBySpec; @@ -694,12 +836,12 @@ public boolean isGroupByLimit() public boolean isUnlimited() { - return groupLimit == NO_LIMIT && groupPerPartitionLimit == NO_LIMIT && rowLimit == NO_LIMIT; + return groupLimit == NO_LIMIT && groupPerPartitionLimit == NO_LIMIT && super.isUnlimited(); } public DataLimits forShortReadRetry(int toFetch) { - return new CQLLimits(toFetch); + return new CQLLimits(NO_LIMIT, toFetch, NO_LIMIT, false); } @Override @@ -712,21 +854,35 @@ public float estimateTotalResults(ColumnFamilyStore cfs) } @Override - public DataLimits forPaging(int pageSize) + public DataLimits forPaging(PageSize pageSize) { - return new CQLGroupByLimits(pageSize, + if (logger.isTraceEnabled()) + logger.trace("{} forPaging({})", hashCode(), pageSize); + + return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, - rowLimit, + pageSize.minBytesCount(bytesLimit), + pageSize.minRowsCount(rowLimit), groupBySpec, state); } @Override - public DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) - { - return new CQLGroupByPagingLimits(pageSize, + public DataLimits forPaging(PageSize pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) + { + if (logger.isTraceEnabled()) + logger.trace("{} forPaging({}, {}, {}) vs state {}/{}", + hashCode(), + pageSize, + lastReturnedKey == null ? "null" : ByteBufferUtil.bytesToHex(lastReturnedKey), + lastReturnedKeyRemaining, + state.partitionKey() == null ? "null" : ByteBufferUtil.bytesToHex(state.partitionKey()), + state.clustering() == null ? "null" : state.clustering().toString()); + + return new CQLGroupByPagingLimits(groupLimit, groupPerPartitionLimit, - rowLimit, + pageSize.minBytesCount(bytesLimit), + pageSize.minRowsCount(rowLimit), groupBySpec, state, lastReturnedKey, @@ -736,8 +892,9 @@ public DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastRe @Override public DataLimits forGroupByInternalPaging(GroupingState state) { - return new CQLGroupByLimits(rowLimit, + return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, + bytesLimit, rowLimit, groupBySpec, state); @@ -769,41 +926,44 @@ public DataLimits withoutState() { return state == GroupingState.EMPTY_STATE ? this - : new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, rowLimit, groupBySpec); + : new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec); } + @Override + public DataLimits withCountedLimit(int newCountedLimit) + { + return new CQLGroupByLimits(newCountedLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec, state); + } + + @Override + public DataLimits withBytesLimit(int bytesLimit) + { + return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec, state); + } + + + @Override public String toString() { - StringBuilder sb = new StringBuilder(); + List limits = new ArrayList<>(4); if (groupLimit != NO_LIMIT) - { - sb.append("GROUP LIMIT ").append(groupLimit); - if (groupPerPartitionLimit != NO_LIMIT || rowLimit != NO_LIMIT) - sb.append(' '); - } - + limits.add("GROUP LIMIT " + groupLimit); if (groupPerPartitionLimit != NO_LIMIT) - { - sb.append("GROUP PER PARTITION LIMIT ").append(groupPerPartitionLimit); - if (rowLimit != NO_LIMIT) - sb.append(' '); - } - + limits.add("GROUP PER PARTITION LIMIT " + groupPerPartitionLimit); + if (bytesLimit != NO_LIMIT) + limits.add("BYTES LIMIT " + bytesLimit); if (rowLimit != NO_LIMIT) - { - sb.append("LIMIT ").append(rowLimit); - } + limits.add("ROWS LIMIT " + rowLimit); - return sb.toString(); + return String.join(" ", limits); } @Override - public boolean isExhausted(Counter counter) + public boolean isCounterBelowLimits(Counter counter) { - return ((GroupByAwareCounter) counter).rowsCounted < rowLimit - && counter.counted() < groupLimit; + return counter.rowsCounted() < rowLimit && counter.bytesCounted() < bytesLimit && counter.counted() < groupLimit; } protected class GroupByAwareCounter extends Counter @@ -817,6 +977,11 @@ protected class GroupByAwareCounter extends Counter */ protected DecoratedKey currentPartitionKey; + /** + * The number of bytes counted so far. + */ + protected int bytesCounted; + /** * The number of rows counted so far. */ @@ -842,6 +1007,8 @@ protected class GroupByAwareCounter extends Counter protected boolean hasLiveStaticRow; + protected int staticRowBytes; + protected boolean hasReturnedRowsFromCurrentPartition; private GroupByAwareCounter(int nowInSec, @@ -862,6 +1029,10 @@ private GroupByAwareCounter(int nowInSec, @Override public void applyToPartition(DecoratedKey partitionKey, Row staticRow) { + if (logger.isTraceEnabled()) + logger.trace("{} - GroupByAwareCounter.newPartition {} with state {}", hashCode(), + ByteBufferUtil.bytesToHex(partitionKey.getKey()), state.partitionKey() != null ? ByteBufferUtil.bytesToHex(state.partitionKey()) : "null"); + if (partitionKey.getKey().equals(state.partitionKey())) { // The only case were we could have state.partitionKey() equals to the partition key @@ -871,6 +1042,7 @@ public void applyToPartition(DecoratedKey partitionKey, Row staticRow) // the static row if any already, so force hasLiveStaticRow to false so we make sure to not count it // once more. hasLiveStaticRow = false; + staticRowBytes = 0; hasReturnedRowsFromCurrentPartition = true; hasUnfinishedGroup = true; } @@ -894,6 +1066,7 @@ public void applyToPartition(DecoratedKey partitionKey, Row staticRow) } hasReturnedRowsFromCurrentPartition = false; hasLiveStaticRow = !staticRow.isEmpty() && isLive(staticRow); + staticRowBytes = hasLiveStaticRow ? staticRow.dataSize() : 0; } currentPartitionKey = partitionKey; // If we are done we need to preserve the groupInCurrentPartition and rowsCountedInCurrentPartition @@ -908,12 +1081,19 @@ public void applyToPartition(DecoratedKey partitionKey, Row staticRow) @Override protected Row applyToStatic(Row row) { + if (logger.isTraceEnabled()) + logger.trace("{} - GroupByAwareCounter.applyToStatic {}/{}", + hashCode(), + currentPartitionKey != null ? ByteBufferUtil.bytesToHex(currentPartitionKey.getKey()) : "null", + row == null ? "null" : row.clustering().toString()); + // It's possible that we're "done" if the partition we just started bumped the number of groups (in // applyToPartition() above), in which case Transformation will still call this method. In that case, we // want to ignore the static row, it should (and will) be returned with the next page/group if needs be. if (enforceLimits && isDone()) { hasLiveStaticRow = false; // The row has not been returned + staticRowBytes = 0; return Rows.EMPTY_STATIC_ROW; } return row; @@ -922,6 +1102,12 @@ protected Row applyToStatic(Row row) @Override public Row applyToRow(Row row) { + if (logger.isTraceEnabled()) + logger.trace("{} - GroupByAwareCounter.applyToRow {}/{}", + hashCode(), + ByteBufferUtil.bytesToHex(currentPartitionKey.getKey()), + row.clustering().toString()); + // We want to check if the row belongs to a new group even if it has been deleted. The goal being // to minimize the chances of having to go through the same data twice if we detect on the next // non deleted row that we have reached the limit. @@ -946,7 +1132,7 @@ public Row applyToRow(Row row) if (isLive(row)) { hasUnfinishedGroup = true; - incrementRowCount(); + incrementRowCount(bytesLimit != NO_LIMIT ? row.dataSize() : 0); hasReturnedRowsFromCurrentPartition = true; } @@ -965,6 +1151,12 @@ public int countedInCurrentPartition() return groupInCurrentPartition; } + @Override + public int bytesCounted() + { + return bytesCounted; + } + @Override public int rowsCounted() { @@ -977,10 +1169,12 @@ public int rowsCountedInCurrentPartition() return rowsCountedInCurrentPartition; } - protected void incrementRowCount() + protected void incrementRowCount(int rowSize) { rowsCountedInCurrentPartition++; - if (++rowsCounted >= rowLimit) + rowsCounted++; + bytesCounted += rowSize; + if (rowsCounted >= rowLimit || bytesCounted >= bytesLimit) stop(); } @@ -1018,7 +1212,7 @@ public void onPartitionClose() // so count it. if (countPartitionsWithOnlyStaticData && hasLiveStaticRow && !hasReturnedRowsFromCurrentPartition) { - incrementRowCount(); + incrementRowCount(staticRowBytes); incrementGroupCount(); incrementGroupInCurrentPartitionCount(); hasUnfinishedGroup = false; @@ -1035,7 +1229,7 @@ public void onClose() // 2) the end of the data is reached // We know that the end of the data is reached if the group limit has not been reached // and the number of rows counted is smaller than the internal page size. - if (hasUnfinishedGroup && groupCounted < groupLimit && rowsCounted < rowLimit) + if (hasUnfinishedGroup && groupCounted < groupLimit && bytesCounted < bytesLimit && rowsCounted < rowLimit) { incrementGroupCount(); incrementGroupInCurrentPartitionCount(); @@ -1043,6 +1237,13 @@ public void onClose() super.onClose(); } + + @Override + public String toString() + { + return String.format("%s(bytes=%s/%s, rows=%s/%s, partition-rows=%s/%s, groups=%s/%s, partition-groups=%s/%s)", this.getClass().getName(), + bytesCounted(), bytesLimit, rowsCounted(), rowLimit, rowsCountedInCurrentPartition(), perPartitionLimit, groupCounted, groupLimit, groupInCurrentPartition, groupPerPartitionLimit); + } } } @@ -1054,6 +1255,7 @@ private static class CQLGroupByPagingLimits extends CQLGroupByLimits public CQLGroupByPagingLimits(int groupLimit, int groupPerPartitionLimit, + int bytesLimit, int rowLimit, AggregationSpecification groupBySpec, GroupingState state, @@ -1062,6 +1264,7 @@ public CQLGroupByPagingLimits(int groupLimit, { super(groupLimit, groupPerPartitionLimit, + bytesLimit, rowLimit, groupBySpec, state); @@ -1077,13 +1280,13 @@ public Kind kind() } @Override - public DataLimits forPaging(int pageSize) + public DataLimits forPaging(PageSize pageSize) { throw new UnsupportedOperationException(); } @Override - public DataLimits forPaging(int pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) + public DataLimits forPaging(PageSize pageSize, ByteBuffer lastReturnedKey, int lastReturnedKeyRemaining) { throw new UnsupportedOperationException(); } @@ -1104,9 +1307,23 @@ public Counter newCounter(int nowInSec, boolean assumeLiveData, boolean countPar @Override public DataLimits withoutState() { - return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, rowLimit, groupBySpec); + return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec); } + @Override + public DataLimits withCountedLimit(int newCountedLimit) + { + return new CQLGroupByPagingLimits(newCountedLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec, state, lastReturnedKey, lastReturnedKeyRemaining); + } + + @Override + public DataLimits withBytesLimit(int bytesLimit) + { + return new CQLGroupByPagingLimits(groupLimit, groupPerPartitionLimit, bytesLimit, rowLimit, groupBySpec, state, lastReturnedKey, lastReturnedKeyRemaining); + } + + + private class PagingGroupByAwareCounter extends GroupByAwareCounter { private PagingGroupByAwareCounter(int nowInSec, boolean assumeLiveData, boolean countPartitionsWithOnlyStaticData, boolean enforceStrictLiveness) @@ -1117,12 +1334,17 @@ private PagingGroupByAwareCounter(int nowInSec, boolean assumeLiveData, boolean @Override public void applyToPartition(DecoratedKey partitionKey, Row staticRow) { + if (logger.isTraceEnabled()) + logger.trace("{} - CQLGroupByPagingLimits.applyToPartition {}", + hashCode(), ByteBufferUtil.bytesToHex(partitionKey.getKey())); + if (partitionKey.getKey().equals(lastReturnedKey)) { currentPartitionKey = partitionKey; groupInCurrentPartition = groupPerPartitionLimit - lastReturnedKeyRemaining; hasReturnedRowsFromCurrentPartition = true; hasLiveStaticRow = false; + staticRowBytes = 0; hasUnfinishedGroup = state.hasClustering(); } else @@ -1131,6 +1353,16 @@ public void applyToPartition(DecoratedKey partitionKey, Row staticRow) } } } + + @Override + public String toString() + { + return new StringJoiner(", ", CQLGroupByPagingLimits.class.getSimpleName() + "[", "]") + .add("super=" + super.toString()) + .add("lastReturnedKey=" + (lastReturnedKey != null ? ByteBufferUtil.bytesToHex(lastReturnedKey) : null)) + .add("lastReturnedKeyRemaining=" + lastReturnedKeyRemaining) + .toString(); + } } public static class Serializer @@ -1145,6 +1377,8 @@ public void serialize(DataLimits limits, DataOutputPlus out, int version, Cluste CQLLimits cqlLimits = (CQLLimits)limits; out.writeUnsignedVInt(cqlLimits.rowLimit); out.writeUnsignedVInt(cqlLimits.perPartitionLimit); + if (version >= MessagingService.VERSION_41) + out.writeUnsignedVInt(cqlLimits.bytesLimit); out.writeBoolean(cqlLimits.isDistinct); if (limits.kind() == Kind.CQL_PAGING_LIMIT) { @@ -1159,6 +1393,8 @@ public void serialize(DataLimits limits, DataOutputPlus out, int version, Cluste out.writeUnsignedVInt(groupByLimits.groupLimit); out.writeUnsignedVInt(groupByLimits.groupPerPartitionLimit); out.writeUnsignedVInt(groupByLimits.rowLimit); + if (version >= MessagingService.VERSION_41) + out.writeUnsignedVInt(groupByLimits.bytesLimit); AggregationSpecification groupBySpec = groupByLimits.groupBySpec; AggregationSpecification.serializer.serialize(groupBySpec, out, version); @@ -1185,12 +1421,13 @@ public DataLimits deserialize(DataInputPlus in, int version, ClusteringComparato { int rowLimit = (int) in.readUnsignedVInt(); int perPartitionLimit = (int) in.readUnsignedVInt(); + int bytesLimit = version >= MessagingService.VERSION_41 ? (int) in.readUnsignedVInt() : NO_LIMIT; boolean isDistinct = in.readBoolean(); if (kind == Kind.CQL_LIMIT) - return cqlLimits(rowLimit, perPartitionLimit, isDistinct); + return cqlLimits(bytesLimit, rowLimit, perPartitionLimit, isDistinct); ByteBuffer lastKey = ByteBufferUtil.readWithVIntLength(in); int lastRemaining = (int) in.readUnsignedVInt(); - return new CQLPagingLimits(rowLimit, perPartitionLimit, isDistinct, lastKey, lastRemaining); + return new CQLPagingLimits(bytesLimit, rowLimit, perPartitionLimit, isDistinct, lastKey, lastRemaining); } case CQL_GROUP_BY_LIMIT: case CQL_GROUP_BY_PAGING_LIMIT: @@ -1198,6 +1435,7 @@ public DataLimits deserialize(DataInputPlus in, int version, ClusteringComparato int groupLimit = (int) in.readUnsignedVInt(); int groupPerPartitionLimit = (int) in.readUnsignedVInt(); int rowLimit = (int) in.readUnsignedVInt(); + int bytesLimit = version >= MessagingService.VERSION_41 ? (int) in.readUnsignedVInt() : NO_LIMIT; AggregationSpecification groupBySpec = AggregationSpecification.serializer.deserialize(in, version, comparator); @@ -1206,6 +1444,7 @@ public DataLimits deserialize(DataInputPlus in, int version, ClusteringComparato if (kind == Kind.CQL_GROUP_BY_LIMIT) return new CQLGroupByLimits(groupLimit, groupPerPartitionLimit, + bytesLimit, rowLimit, groupBySpec, state); @@ -1214,6 +1453,7 @@ public DataLimits deserialize(DataInputPlus in, int version, ClusteringComparato int lastRemaining = (int) in.readUnsignedVInt(); return new CQLGroupByPagingLimits(groupLimit, groupPerPartitionLimit, + bytesLimit, rowLimit, groupBySpec, state, @@ -1234,6 +1474,8 @@ public long serializedSize(DataLimits limits, int version, ClusteringComparator CQLLimits cqlLimits = (CQLLimits) limits; size += TypeSizes.sizeofUnsignedVInt(cqlLimits.rowLimit); size += TypeSizes.sizeofUnsignedVInt(cqlLimits.perPartitionLimit); + if (version >= MessagingService.VERSION_41) + size += TypeSizes.sizeofUnsignedVInt(cqlLimits.bytesLimit); size += TypeSizes.sizeof(cqlLimits.isDistinct); if (limits.kind() == Kind.CQL_PAGING_LIMIT) { @@ -1248,6 +1490,8 @@ public long serializedSize(DataLimits limits, int version, ClusteringComparator size += TypeSizes.sizeofUnsignedVInt(groupByLimits.groupLimit); size += TypeSizes.sizeofUnsignedVInt(groupByLimits.groupPerPartitionLimit); size += TypeSizes.sizeofUnsignedVInt(groupByLimits.rowLimit); + if (version >= MessagingService.VERSION_41) + size += TypeSizes.sizeofUnsignedVInt(groupByLimits.bytesLimit); AggregationSpecification groupBySpec = groupByLimits.groupBySpec; size += AggregationSpecification.serializer.serializedSize(groupBySpec, version); diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java index 68a1d57fe1ac..5885ef8d3b68 100644 --- a/src/java/org/apache/cassandra/db/filter/RowFilter.java +++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java @@ -23,12 +23,18 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import com.google.common.base.Objects; +import com.google.common.base.Predicate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.restrictions.CustomIndexExpression; +import org.apache.cassandra.cql3.restrictions.Restrictions; +import org.apache.cassandra.cql3.restrictions.StatementRestrictions; import org.apache.cassandra.db.*; import org.apache.cassandra.db.context.*; import org.apache.cassandra.db.marshal.*; @@ -37,8 +43,10 @@ import org.apache.cassandra.db.rows.*; import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.IndexRegistry; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.TableMetadata; @@ -62,56 +70,23 @@ public abstract class RowFilter implements Iterable private static final Logger logger = LoggerFactory.getLogger(RowFilter.class); public static final Serializer serializer = new Serializer(); - public static final RowFilter NONE = new CQLFilter(Collections.emptyList()); + public static final RowFilter NONE = CQLFilter.NONE; - protected final List expressions; + protected final FilterElement root; - protected RowFilter(List expressions) + protected RowFilter(FilterElement root) { - this.expressions = expressions; + this.root = root; } - public static RowFilter create() + public FilterElement root() { - return new CQLFilter(new ArrayList<>()); - } - - public static RowFilter create(int capacity) - { - return new CQLFilter(new ArrayList<>(capacity)); - } - - public SimpleExpression add(ColumnMetadata def, Operator op, ByteBuffer value) - { - SimpleExpression expression = new SimpleExpression(def, op, value); - add(expression); - return expression; - } - - public void addMapEquality(ColumnMetadata def, ByteBuffer key, Operator op, ByteBuffer value) - { - add(new MapEqualityExpression(def, key, op, value)); - } - - public void addCustomIndexExpression(TableMetadata metadata, IndexMetadata targetIndex, ByteBuffer value) - { - add(new CustomExpression(metadata, targetIndex, value)); - } - - private void add(Expression expression) - { - expression.validate(); - expressions.add(expression); - } - - public void addUserExpression(UserExpression e) - { - expressions.add(e); + return root; } public List getExpressions() { - return expressions; + return root.expressions; } /** @@ -120,7 +95,7 @@ public List getExpressions() */ public boolean hasExpressionOnClusteringOrRegularColumns() { - for (Expression expression : expressions) + for (Expression expression : root) { ColumnMetadata column = expression.column(); if (column.isClusteringColumn() || column.isRegular()) @@ -141,7 +116,7 @@ public boolean hasExpressionOnClusteringOrRegularColumns() */ public UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int nowInSec) { - return expressions.isEmpty() ? iter : Transformation.apply(iter, filter(iter.metadata(), nowInSec)); + return root.isEmpty() ? iter : Transformation.apply(iter, filter(iter.metadata(), nowInSec)); } /** @@ -154,7 +129,7 @@ public UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int */ public PartitionIterator filter(PartitionIterator iter, TableMetadata metadata, int nowInSec) { - return expressions.isEmpty() ? iter : Transformation.apply(iter, filter(metadata, nowInSec)); + return root.isEmpty() ? iter : Transformation.apply(iter, filter(metadata, nowInSec)); } /** @@ -171,14 +146,9 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, // We purge all tombstones as the expressions isSatisfiedBy methods expects it Row purged = row.purge(DeletionPurger.PURGE_ALL, nowInSec, metadata.enforceStrictLiveness()); if (purged == null) - return expressions.isEmpty(); + return root.isEmpty(); - for (Expression e : expressions) - { - if (!e.isSatisfiedBy(metadata, partitionKey, purged)) - return false; - } - return true; + return root.isSatisfiedBy(metadata, partitionKey, purged); } /** @@ -187,7 +157,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, */ public boolean partitionKeyRestrictionsAreSatisfiedBy(DecoratedKey key, AbstractType keyValidator) { - for (Expression e : expressions) + for (Expression e : root) { if (!e.column.isPartitionKey()) continue; @@ -207,7 +177,7 @@ public boolean partitionKeyRestrictionsAreSatisfiedBy(DecoratedKey key, Abstract */ public boolean clusteringKeyRestrictionsAreSatisfiedBy(Clustering clustering) { - for (Expression e : expressions) + for (Expression e : root) { if (!e.column.isClusteringColumn()) continue; @@ -226,69 +196,329 @@ public boolean clusteringKeyRestrictionsAreSatisfiedBy(Clustering clustering) */ public RowFilter without(Expression expression) { - assert expressions.contains(expression); - if (expressions.size() == 1) + assert root.contains(expression); + if (root.size() == 1) return RowFilter.NONE; - List newExpressions = new ArrayList<>(expressions.size() - 1); - for (Expression e : expressions) - if (!e.equals(expression)) - newExpressions.add(e); - - return withNewExpressions(newExpressions); + return new CQLFilter(root.filter(e -> !e.equals(expression))); } public RowFilter withoutExpressions() { - return withNewExpressions(Collections.emptyList()); + return NONE; } - protected abstract RowFilter withNewExpressions(List expressions); + public RowFilter restrict(Predicate filter) + { + return new CQLFilter(root.filter(filter)); + } public boolean isEmpty() { - return expressions.isEmpty(); + return root.isEmpty(); } public Iterator iterator() { - return expressions.iterator(); + return root.iterator(); } @Override public String toString() { - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < expressions.size(); i++) + return root.toString(); + } + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + { + private FilterElement.Builder current = new FilterElement.Builder(false); + + public RowFilter build() { - if (i > 0) - sb.append(" AND "); - sb.append(expressions.get(i)); + return new CQLFilter(current.build()); + } + + public RowFilter buildFromRestrictions(StatementRestrictions restrictions, IndexRegistry indexManager, TableMetadata table, QueryOptions options) + { + return new CQLFilter(doBuild(restrictions, indexManager, table, options)); + } + + private FilterElement doBuild(StatementRestrictions restrictions, IndexRegistry indexManager, TableMetadata table, QueryOptions options) + { + FilterElement.Builder element = new FilterElement.Builder(restrictions.isDisjunction()); + this.current = element; + + for (Restrictions restrictionSet : restrictions.filterRestrictions().getRestrictions()) + restrictionSet.addToRowFilter(this, indexManager, options); + + for (CustomIndexExpression expression : restrictions.filterRestrictions().getExternalExpressions()) + expression.addToRowFilter(this, table, options); + + for (StatementRestrictions child : restrictions.children()) + element.children.add(doBuild(child, indexManager, table, options)); + + return element.build(); + } + + public SimpleExpression add(ColumnMetadata def, Operator op, ByteBuffer value) + { + SimpleExpression expression = new SimpleExpression(def, op, value); + add(expression); + return expression; + } + + public void addMapEquality(ColumnMetadata def, ByteBuffer key, Operator op, ByteBuffer value) + { + add(new MapEqualityExpression(def, key, op, value)); + } + + public void addCustomIndexExpression(TableMetadata metadata, IndexMetadata targetIndex, ByteBuffer value) + { + add(CustomExpression.build(metadata, targetIndex, value)); + } + + private void add(Expression expression) + { + expression.validate(); + current.expressions.add(expression); + } + + public void addUserExpression(UserExpression e) + { + current.expressions.add(e); } - return sb.toString(); } - private static class CQLFilter extends RowFilter + public static class FilterElement implements Iterable { - private CQLFilter(List expressions) + public static final Serializer serializer = new Serializer(); + + public static final FilterElement NONE = new FilterElement(false, Collections.emptyList(), Collections.emptyList()); + + private boolean isDisjunction; + + private final List expressions; + + private final List children; + + public FilterElement(boolean isDisjunction, List expressions, List children) { - super(expressions); + this.isDisjunction = isDisjunction; + this.expressions = expressions; + this.children = children; } - protected Transformation> filter(TableMetadata metadata, int nowInSec) + public boolean isDisjunction() + { + return isDisjunction; + } + + public List expressions() + { + return expressions; + } + + public Iterator iterator() + { + List allExpressions = new ArrayList<>(expressions); + for (FilterElement child : children) + allExpressions.addAll(child.expressions); + return allExpressions.iterator(); + } + + public FilterElement filter(Predicate filter) + { + FilterElement.Builder builder = new Builder(isDisjunction); + + expressions.stream().filter(filter).forEach(e -> builder.expressions.add(e)); + + children.stream().map(c -> c.filter(filter)).forEach(c -> builder.children.add(c)); + + return builder.build(); + } + + public List children() + { + return children; + } + + public boolean isEmpty() + { + return expressions.isEmpty() && children.isEmpty(); + } + + public boolean contains(Expression expression) { - List partitionLevelExpressions = new ArrayList<>(); - List rowLevelExpressions = new ArrayList<>(); - for (Expression e: expressions) + return expressions.contains(expression) || children.stream().anyMatch(c -> contains(expression)); + } + + public FilterElement partitionLevelTree() + { + return new FilterElement(isDisjunction, + expressions.stream() + .filter(e -> e.column.isStatic() || e.column.isPartitionKey()) + .collect(Collectors.toList()), + children.stream() + .map(FilterElement::partitionLevelTree) + .collect(Collectors.toList())); + } + + public FilterElement rowLevelTree() + { + return new FilterElement(isDisjunction, + expressions.stream() + .filter(e -> !e.column.isStatic() && !e.column.isPartitionKey()) + .collect(Collectors.toList()), + children.stream() + .map(FilterElement::rowLevelTree) + .collect(Collectors.toList())); + } + + public int size() + { + return expressions.size() + children.stream().mapToInt(FilterElement::size).sum(); + } + + public boolean isSatisfiedBy(TableMetadata table, DecoratedKey key, Row row) + { + if (isEmpty()) + return true; + if (isDisjunction) + { + for (Expression e : expressions) + if (e.isSatisfiedBy(table, key, row)) + return true; + for (FilterElement child : children) + if (child.isSatisfiedBy(table, key, row)) + return true; + return false; + } + else + { + for (Expression e : expressions) + if (!e.isSatisfiedBy(table, key, row)) + return false; + for (FilterElement child : children) + if (!child.isSatisfiedBy(table, key, row)) + return false; + return true; + } + } + + @Override + public String toString() + { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < expressions.size(); i++) + { + if (sb.length() > 0) + sb.append(isDisjunction ? " OR " : " AND "); + sb.append(expressions.get(i)); + } + for (int i = 0; i < children.size(); i++) + { + if (sb.length() > 0) + sb.append(isDisjunction ? " OR " : " AND "); + sb.append("("); + sb.append(children.get(i)); + sb.append(")"); + } + return sb.toString(); + } + + public static class Builder + { + private boolean isDisjunction; + private final List expressions = new ArrayList<>(); + private final List children = new ArrayList<>(); + + public Builder(boolean isDisjunction) + { + this.isDisjunction = isDisjunction; + } + + public FilterElement build() + { + return new FilterElement(isDisjunction, expressions, children); + } + } + + public static class Serializer + { + public void serialize(FilterElement operation, DataOutputPlus out, int version) throws IOException { - if (e.column.isStatic() || e.column.isPartitionKey()) - partitionLevelExpressions.add(e); - else - rowLevelExpressions.add(e); + assert (!operation.isDisjunction && operation.children().isEmpty()) || version == MessagingService.VERSION_SG_10 : + "Attempting to serialize a disjunct row filter to a node that doesn't support disjunction"; + + out.writeUnsignedVInt(operation.expressions.size()); + for (Expression expr : operation.expressions) + Expression.serializer.serialize(expr, out, version); + + if (version < MessagingService.VERSION_SG_10) + return; + + out.writeBoolean(operation.isDisjunction); + out.writeUnsignedVInt(operation.children.size()); + for (FilterElement child : operation.children) + serialize(child, out, version); } - long numberOfRegularColumnExpressions = rowLevelExpressions.size(); - final boolean filterNonStaticColumns = numberOfRegularColumnExpressions > 0; + public FilterElement deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException + { + int size = (int)in.readUnsignedVInt(); + List expressions = new ArrayList<>(size); + for (int i = 0; i < size; i++) + expressions.add(Expression.serializer.deserialize(in, version, metadata)); + + if (version < MessagingService.VERSION_SG_10) + return new FilterElement(false, expressions, Collections.emptyList()); + + boolean isDisjunction = in.readBoolean(); + size = (int)in.readUnsignedVInt(); + List children = new ArrayList<>(size); + for (int i = 0; i < size; i++) + children.add(deserialize(in, version, metadata)); + return new FilterElement(isDisjunction, expressions, children); + } + + public long serializedSize(FilterElement operation, int version) + { + long size = TypeSizes.sizeofUnsignedVInt(operation.expressions.size()); + for (Expression expr : operation.expressions) + size += Expression.serializer.serializedSize(expr, version); + + if (version < MessagingService.VERSION_SG_10) + return size; + + size++; // isDisjunction boolean + size += TypeSizes.sizeofUnsignedVInt(operation.children.size()); + for (FilterElement child : operation.children) + size += serializedSize(child, version); + return size; + } + } + } + + private static class CQLFilter extends RowFilter + { + private static final CQLFilter NONE = new CQLFilter(FilterElement.NONE); + + private CQLFilter(FilterElement operation) + { + super(operation); + } + + protected Transformation> filter(TableMetadata metadata, int nowInSec) + { + FilterElement partitionLevelOperation = root.partitionLevelTree(); + FilterElement rowLevelOperation = root.rowLevelTree(); + + final boolean filterNonStaticColumns = rowLevelOperation.size() > 0; return new Transformation>() { @@ -300,12 +530,11 @@ protected BaseRowIterator applyToPartition(BaseRowIterator partition) pk = partition.partitionKey(); // Short-circuit all partitions that won't match based on static and partition keys - for (Expression e : partitionLevelExpressions) - if (!e.isSatisfiedBy(metadata, partition.partitionKey(), partition.staticRow())) - { - partition.close(); - return null; - } + if (!partitionLevelOperation.isSatisfiedBy(metadata, partition.partitionKey(), partition.staticRow())) + { + partition.close(); + return null; + } BaseRowIterator iterator = partition instanceof UnfilteredRowIterator ? Transformation.apply((UnfilteredRowIterator) partition, this) @@ -326,24 +555,18 @@ public Row applyToRow(Row row) if (purged == null) return null; - for (Expression e : rowLevelExpressions) - if (!e.isSatisfiedBy(metadata, pk, purged)) - return null; + if (!rowLevelOperation.isSatisfiedBy(metadata, pk, purged)) + return null; return row; } }; } - - protected RowFilter withNewExpressions(List expressions) - { - return new CQLFilter(expressions); - } } public static abstract class Expression { - private static final Serializer serializer = new Serializer(); + public static final Serializer serializer = new Serializer(); // Note: the order of this enum matter, it's used for serialization, // and this is why we have some UNUSEDX for values we don't use anymore @@ -478,7 +701,7 @@ public int hashCode() return Objects.hashCode(column.name, operator, value); } - private static class Serializer + public static class Serializer { public void serialize(Expression expression, DataOutputPlus out, int version) throws IOException { @@ -522,9 +745,9 @@ public Expression deserialize(DataInputPlus in, int version, TableMetadata metad // custom expressions (3.0+ only) do not contain a column or operator, only a value if (kind == Kind.CUSTOM) { - return new CustomExpression(metadata, - IndexMetadata.serializer.deserialize(in, version, metadata), - ByteBufferUtil.readWithShortLength(in)); + return CustomExpression.build(metadata, + IndexMetadata.serializer.deserialize(in, version, metadata), + ByteBufferUtil.readWithShortLength(in)); } if (kind == Kind.USER) @@ -589,7 +812,7 @@ public long serializedSize(Expression expression, int version) */ public static class SimpleExpression extends Expression { - SimpleExpression(ColumnMetadata column, Operator operator, ByteBuffer value) + public SimpleExpression(ColumnMetadata column, Operator operator, ByteBuffer value) { super(column, operator, value); } @@ -603,6 +826,7 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, switch (operator) { case EQ: + case IN: case LT: case LTE: case GTE: @@ -695,11 +919,6 @@ public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, ByteBuffer foundValue = getValue(metadata, partitionKey, row); return foundValue != null && mapType.getSerializer().getSerializedValue(foundValue, value, mapType.getKeysType()) != null; } - - case IN: - // It wouldn't be terribly hard to support this (though doing so would imply supporting - // IN for 2ndary index) but currently we don't. - throw new AssertionError(); } throw new AssertionError(); } @@ -739,7 +958,7 @@ protected Kind kind() * An expression of the form 'column' ['key'] = 'value' (which is only * supported when 'column' is a map). */ - private static class MapEqualityExpression extends Expression + public static class MapEqualityExpression extends Expression { private final ByteBuffer key; @@ -833,7 +1052,7 @@ protected Kind kind() * A custom index expression for use with 2i implementations which support custom syntax and which are not * necessarily linked to a single column in the base table. */ - public static final class CustomExpression extends Expression + public static class CustomExpression extends Expression { private final IndexMetadata targetIndex; private final TableMetadata table; @@ -846,6 +1065,12 @@ public CustomExpression(TableMetadata table, IndexMetadata targetIndex, ByteBuff this.table = table; } + public static CustomExpression build(TableMetadata metadata, IndexMetadata targetIndex, ByteBuffer value) + { + // delegate the expression creation to the target custom index + return Keyspace.openAndGetStore(metadata).indexManager.getIndex(targetIndex).customExpressionFor(metadata, value); + } + private static ColumnMetadata makeDefinition(TableMetadata table, IndexMetadata index) { // Similarly to how we handle non-defined columns in thift, we create a fake column definition to @@ -984,29 +1209,20 @@ public static class Serializer public void serialize(RowFilter filter, DataOutputPlus out, int version) throws IOException { out.writeBoolean(false); // Old "is for thrift" boolean - out.writeUnsignedVInt(filter.expressions.size()); - for (Expression expr : filter.expressions) - Expression.serializer.serialize(expr, out, version); - + FilterElement.serializer.serialize(filter.root, out, version); } public RowFilter deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException { in.readBoolean(); // Unused - int size = (int)in.readUnsignedVInt(); - List expressions = new ArrayList<>(size); - for (int i = 0; i < size; i++) - expressions.add(Expression.serializer.deserialize(in, version, metadata)); - - return new CQLFilter(expressions); + FilterElement operation = FilterElement.serializer.deserialize(in, version, metadata); + return new CQLFilter(operation); } public long serializedSize(RowFilter filter, int version) { long size = 1 // unused boolean - + TypeSizes.sizeofUnsignedVInt(filter.expressions.size()); - for (Expression expr : filter.expressions) - size += Expression.serializer.serializedSize(expr, version); + + FilterElement.serializer.serializedSize(filter.root, version); return size; } } diff --git a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java index 28d49ae370c5..d59acbc77676 100644 --- a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java +++ b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java @@ -26,7 +26,7 @@ public class TombstoneOverwhelmingException extends RuntimeException { - public TombstoneOverwhelmingException(int numTombstones, String query, TableMetadata metadata, DecoratedKey lastPartitionKey, ClusteringPrefix lastClustering) + public TombstoneOverwhelmingException(long numTombstones, String query, TableMetadata metadata, DecoratedKey lastPartitionKey, ClusteringPrefix lastClustering) { super(String.format("Scanned over %d tombstones during query '%s' (last scanned row token was %s and partion key was (%s)); query aborted", numTombstones, query, lastPartitionKey.getToken(), makePKString(metadata, lastPartitionKey.getKey(), lastClustering))); diff --git a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java index 8e0d5144b8af..336a5774de9f 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java +++ b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java @@ -22,6 +22,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.*; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.utils.Throwables; @@ -40,7 +41,7 @@ class Helpers * really present, and that the items to add are not (unless we're also removing them) * @return a new set with the contents of the provided one modified */ - static Set replace(Set original, Set remove, Iterable add) + static Set replace(Set original, Set remove, Iterable add) { return ImmutableSet.copyOf(replace(identityMap(original), remove, add).keySet()); } @@ -50,11 +51,11 @@ static Set replace(Set original, Set remove, Iterable add) * really present, and that the items to add are not (unless we're also removing them) * @return a new identity map with the contents of the provided one modified */ - static Map replace(Map original, Set remove, Iterable add) + static Map replace(Map original, Set remove, Iterable add) { // ensure the ones being removed are the exact same ones present for (T reader : remove) - assert original.get(reader) == reader; + assert original.get(reader) == reader : String.format("%s not found in original set: %s", reader, original); // ensure we don't already contain any we're adding, that we aren't also removing assert !any(add, and(not(in(remove)), in(original.keySet()))) : String.format("original:%s remove:%s add:%s", original.keySet(), remove, add); @@ -72,10 +73,10 @@ static Map replace(Map original, Set remove, Iterable add) * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise */ - static void setupOnline(Iterable readers) + static void setupOnline(ColumnFamilyStore cfs, Iterable readers) { for (SSTableReader reader : readers) - reader.setupOnline(); + reader.setupOnline(cfs); } /** @@ -126,14 +127,18 @@ static Throwable markObsolete(List obsoletions, Throw return accumulate; } - static Throwable prepareForObsoletion(Iterable readers, LogTransaction txnLogs, List obsoletions, Throwable accumulate) + static Throwable prepareForObsoletion(Iterable readers, + LogTransaction txnLogs, + List obsoletions, + Tracker tracker, + Throwable accumulate) { Map logRecords = txnLogs.makeRemoveRecords(readers); for (SSTableReader reader : readers) { try { - obsoletions.add(new LogTransaction.Obsoletion(reader, txnLogs.obsoleted(reader, logRecords.get(reader)))); + obsoletions.add(new LogTransaction.Obsoletion(reader, txnLogs.obsoleted(reader, logRecords.get(reader), tracker))); } catch (Throwable t) { diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java index 574c6a449936..720f1ebf93eb 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java @@ -28,13 +28,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.io.sstable.format.SSTableReader.UniqueIdentifier; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.io.sstable.format.SSTableReader.UniqueIdentifier; import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.Transactional; @@ -103,7 +103,7 @@ public String toString() } } - public final Tracker tracker; + private final Tracker tracker; // The transaction logs keep track of new and old sstable files private final LogTransaction log; // the original readers this transaction was opened over, and that it guards @@ -142,7 +142,7 @@ public static LifecycleTransaction offline(OperationType operationType, SSTableR public static LifecycleTransaction offline(OperationType operationType, Iterable readers) { // if offline, for simplicity we just use a dummy tracker - Tracker dummy = new Tracker(null, false); + Tracker dummy = Tracker.newDummyTracker(); dummy.addInitialSSTables(readers); dummy.apply(updateCompacting(emptySet(), readers)); return new LifecycleTransaction(dummy, operationType, readers); @@ -154,17 +154,17 @@ public static LifecycleTransaction offline(OperationType operationType, Iterable @SuppressWarnings("resource") // log closed during postCleanup public static LifecycleTransaction offline(OperationType operationType) { - Tracker dummy = new Tracker(null, false); - return new LifecycleTransaction(dummy, new LogTransaction(operationType, dummy), Collections.emptyList()); + Tracker dummy = Tracker.newDummyTracker(); + return new LifecycleTransaction(dummy, new LogTransaction(operationType), Collections.emptyList()); } @SuppressWarnings("resource") // log closed during postCleanup - LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable readers) + LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable readers) { - this(tracker, new LogTransaction(operationType, tracker), readers); + this(tracker, new LogTransaction(operationType), readers); } - LifecycleTransaction(Tracker tracker, LogTransaction log, Iterable readers) + LifecycleTransaction(Tracker tracker, LogTransaction log, Iterable readers) { this.tracker = tracker; this.log = log; @@ -192,6 +192,11 @@ public UUID opId() return log.id(); } + public Set getCompacting() + { + return tracker.getCompacting(); + } + public void doPrepare() { // note for future: in anticompaction two different operations use the same Transaction, and both prepareToCommit() @@ -202,7 +207,7 @@ public void doPrepare() // prepare for compaction obsolete readers as long as they were part of the original set // since those that are not original are early readers that share the same desc with the finals - maybeFail(prepareForObsoletion(filterIn(logged.obsolete, originals), log, obsoletions = new ArrayList<>(), null)); + maybeFail(prepareForObsoletion(filterIn(logged.obsolete, originals), log, obsoletions = new ArrayList<>(), tracker, null)); log.prepareToCommit(); } @@ -253,7 +258,7 @@ public Throwable doAbort(Throwable accumulate) Iterable obsolete = filterOut(concatUniq(staged.update, logged.update), originals); logger.trace("Obsoleting {}", obsolete); - accumulate = prepareForObsoletion(obsolete, log, obsoletions = new ArrayList<>(), accumulate); + accumulate = prepareForObsoletion(obsolete, log, obsoletions = new ArrayList<>(), tracker, accumulate); // it's safe to abort even if committed, see maybeFail in doCommit() above, in this case it will just report // a failure to abort, which is useful information to have for debug accumulate = log.abort(accumulate); @@ -316,6 +321,12 @@ public boolean isOffline() return tracker.isDummy(); } + @VisibleForTesting + public void unsafeClose() + { + log.close(); + } + /** * call when a consistent batch of changes is ready to be made atomically visible * these will be exposed in the Tracker atomically, or an exception will be thrown; in this case diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java index 0295357e8f0f..6a07392217b7 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java @@ -222,13 +222,28 @@ void printContentsWithAnyErrors(StringBuilder str) */ void append(LogRecord record) { - Throwable err = Throwables.perform(null, replicas().stream().map(r -> () -> r.append(record))); + Throwable err = null; + int failed = 0; + for (LogReplica replica : replicas()) + { + try + { + replica.append(record); + } + catch (Throwable t) + { + logger.warn("Failed to add record to a replica: {}", t.getMessage()); + err = Throwables.merge(err, t); + failed++; + } + } + if (err != null) { - if (!record.isFinal() || err.getSuppressed().length == replicas().size() -1) + if (!record.isFinal() || failed == replicas().size()) Throwables.maybeFail(err); - logger.error("Failed to add record '{}' to some replicas '{}'", record, this); + logger.error("Failed to add record '{}' to some replicas '{}'", record, this, err); } } diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java index 85df4d64e04f..3e4e37639490 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java @@ -28,12 +28,15 @@ import java.util.concurrent.TimeUnit; import java.util.function.Predicate; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.Runnables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.codahale.metrics.Counter; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.Directories; @@ -107,7 +110,6 @@ public CorruptTransactionLogException(String message, LogFile txnFile) } } - private final Tracker tracker; private final LogFile txnFile; // We need an explicit lock because the transaction tidier cannot store a reference to the transaction private final Object lock; @@ -120,12 +122,6 @@ public CorruptTransactionLogException(String message, LogFile txnFile) LogTransaction(OperationType opType) { - this(opType, null); - } - - LogTransaction(OperationType opType, Tracker tracker) - { - this.tracker = tracker; this.txnFile = new LogFile(opType, UUIDGen.getTimeUUID()); this.lock = new Object(); this.selfRef = new Ref<>(this, new TransactionTidier(txnFile, lock)); @@ -165,13 +161,13 @@ void untrackNew(SSTable table) @VisibleForTesting SSTableTidier obsoleted(SSTableReader sstable) { - return obsoleted(sstable, LogRecord.make(Type.REMOVE, sstable)); + return obsoleted(sstable, LogRecord.make(Type.REMOVE, sstable), null); } /** * Schedule a reader for deletion as soon as it is fully unreferenced. */ - SSTableTidier obsoleted(SSTableReader reader, LogRecord logRecord) + SSTableTidier obsoleted(SSTableReader reader, LogRecord logRecord, @Nullable Tracker tracker) { synchronized (lock) { @@ -183,7 +179,7 @@ SSTableTidier obsoleted(SSTableReader reader, LogRecord logRecord) if (txnFile.contains(Type.REMOVE, reader, logRecord)) throw new IllegalArgumentException(); - return new SSTableTidier(reader, true, this); + return new SSTableTidier(reader, true, this, tracker); } txnFile.addRecord(logRecord); @@ -191,7 +187,7 @@ SSTableTidier obsoleted(SSTableReader reader, LogRecord logRecord) if (tracker != null) tracker.notifyDeleting(reader); - return new SSTableTidier(reader, false, this); + return new SSTableTidier(reader, false, this, tracker); } } @@ -349,21 +345,23 @@ public static class SSTableTidier implements Runnable // must not retain a reference to the SSTableReader, else leak detection cannot kick in private final Descriptor desc; private final long sizeOnDisk; - private final Tracker tracker; private final boolean wasNew; private final Object lock; private final Ref parentRef; private final UUID txnId; + private final boolean onlineTxn; + private final Counter totalDiskSpaceUsed; - public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent) + public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction parent, Tracker tracker) { this.desc = referent.descriptor; this.sizeOnDisk = referent.bytesOnDisk(); - this.tracker = parent.tracker; this.wasNew = wasNew; this.lock = parent.lock; this.parentRef = parent.selfRef.tryRef(); this.txnId = parent.id(); + this.onlineTxn = tracker != null && !tracker.isDummy(); + this.totalDiskSpaceUsed = tracker != null && tracker.cfstore != null ? tracker.cfstore.metric.totalDiskSpaceUsed : null; if (this.parentRef == null) throw new IllegalStateException("Transaction already completed"); @@ -371,7 +369,7 @@ public SSTableTidier(SSTableReader referent, boolean wasNew, LogTransaction pare public void run() { - if (tracker != null && !tracker.isDummy()) + if (onlineTxn) SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation); synchronized (lock) @@ -399,8 +397,8 @@ else if (!wasNew) return; } - if (tracker != null && tracker.cfstore != null && !wasNew) - tracker.cfstore.metric.totalDiskSpaceUsed.dec(sizeOnDisk); + if (totalDiskSpaceUsed != null && !wasNew) + totalDiskSpaceUsed.dec(sizeOnDisk); // release the referent to the parent so that the all transaction files can be released parentRef.release(); @@ -548,7 +546,9 @@ static boolean removeUnfinishedLeftovers(Map.Entry> entry) } else { - logger.error("Unexpected disk state: failed to read transaction log {}", txn.toString(true)); + logger.error("Unexpected disk state: failed to read transaction log {}, " + + "check logs before last shutdown for any errors, and ensure txn log files were not edited manually.", + txn.toString(true)); return false; } } diff --git a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java index 61fab98a0082..9315e8b35176 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java +++ b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java @@ -54,7 +54,7 @@ public static List> buildIntervals(It { List> intervals = new ArrayList<>(Iterables.size(sstables)); for (SSTableReader sstable : sstables) - intervals.add(Interval.create(sstable.first, sstable.last, sstable)); + intervals.add(Interval.create(sstable.getFirst(), sstable.getLast(), sstable)); return intervals; } } diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java index 3d72a113b804..3c2b4b586434 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java +++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java @@ -30,7 +30,7 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Directories; -import org.apache.cassandra.db.Memtable; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.commitlog.CommitLogPosition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,17 +74,23 @@ public class Tracker public final boolean loadsstables; /** + * @param columnFamilyStore * @param memtable Initial Memtable. Can be null. * @param loadsstables true to indicate to load SSTables (TODO: remove as this is only accessed from 2i) */ - public Tracker(Memtable memtable, boolean loadsstables) + public Tracker(ColumnFamilyStore columnFamilyStore, Memtable memtable, boolean loadsstables) { - this.cfstore = memtable != null ? memtable.cfs : null; + this.cfstore = columnFamilyStore; this.view = new AtomicReference<>(); this.loadsstables = loadsstables; this.reset(memtable); } + public static Tracker newDummyTracker() + { + return new Tracker(null, null, false); + } + public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType) { return tryModify(singleton(sstable), operationType); @@ -93,7 +99,7 @@ public LifecycleTransaction tryModify(SSTableReader sstable, OperationType opera /** * @return a Transaction over the provided sstables if we are able to mark the given @param sstables as compacted, before anyone else */ - public LifecycleTransaction tryModify(Iterable sstables, OperationType operationType) + public LifecycleTransaction tryModify(Iterable sstables, OperationType operationType) { if (Iterables.isEmpty(sstables)) return new LifecycleTransaction(this, operationType, sstables); @@ -182,6 +188,12 @@ Throwable updateSizeTracking(Iterable oldSSTables, Iterable sstables) @@ -210,7 +222,7 @@ private void addSSTablesInternal(Iterable sstables, boolean updateSize) { if (!isDummy()) - setupOnline(sstables); + setupOnline(cfstore, sstables); apply(updateLiveSet(emptySet(), sstables)); if(updateSize) maybeFail(updateSizeTracking(emptySet(), sstables, null)); @@ -252,7 +264,7 @@ public Throwable dropSSTables(Throwable accumulate) */ public Throwable dropSSTables(final Predicate remove, OperationType operationType, Throwable accumulate) { - try (LogTransaction txnLogs = new LogTransaction(operationType, this)) + try (LogTransaction txnLogs = new LogTransaction(operationType)) { Pair result = apply(view -> { Set toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting)))); @@ -265,7 +277,7 @@ public Throwable dropSSTables(final Predicate remove, OperationTy // It is important that any method accepting/returning a Throwable never throws an exception, and does its best // to complete the instructions given to it List obsoletions = new ArrayList<>(); - accumulate = prepareForObsoletion(removed, txnLogs, obsoletions, accumulate); + accumulate = prepareForObsoletion(removed, txnLogs, obsoletions, this, accumulate); try { txnLogs.finish(); @@ -361,7 +373,7 @@ public void replaceFlushed(Memtable memtable, Iterable sstables) return; } - sstables.forEach(SSTableReader::setupOnline); + setupOnline(cfstore, sstables); // back up before creating a new Snapshot (which makes the new one eligible for compaction) maybeIncrementallyBackup(sstables); @@ -390,14 +402,19 @@ public Set getCompacting() return view.get().compacting; } - public Iterable getUncompacting() + public Iterable getNoncompacting() { return view.get().select(SSTableSet.NONCOMPACTING); } - public Iterable getUncompacting(Iterable candidates) + public Iterable getNoncompacting(Iterable candidates) { - return view.get().getUncompacting(candidates); + return view.get().getNoncompacting(candidates); + } + + public Set getLiveSSTables() + { + return view.get().liveSSTables(); } public void maybeIncrementallyBackup(final Iterable sstables) @@ -516,6 +533,8 @@ public boolean isDummy() public void subscribe(INotificationConsumer consumer) { subscribers.add(consumer); + if (logger.isTraceEnabled()) + logger.trace("{} subscribed to the data tracker.", consumer); } public void unsubscribe(INotificationConsumer consumer) @@ -536,6 +555,12 @@ public View getView() @VisibleForTesting public void removeUnsafe(Set toRemove) { - Pair result = apply(view -> updateLiveSet(toRemove, emptySet()).apply(view)); + apply(view -> updateLiveSet(toRemove, emptySet()).apply(view)); + } + + @VisibleForTesting + public void removeCompactingUnsafe(Set toRemove) + { + apply(view -> updateCompacting(toRemove, emptySet()).apply(view)); } } diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java index b26426de63ae..79113c3163e1 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/View.java +++ b/src/java/org/apache/cassandra/db/lifecycle/View.java @@ -26,7 +26,7 @@ import com.google.common.collect.*; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.Memtable; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -136,13 +136,30 @@ public Iterable select(SSTableSet sstableSet) case NONCOMPACTING: return filter(sstables, (s) -> !compacting.contains(s)); case CANONICAL: + // When early open is not in play, the LIVE and CANONICAL sets are the same. + // However, when we do have early-open sstables, we will have some unfinished sources in the live set. + // For these sources we need to extract the originals, in their non-moved-start versions, from the + // compacting set. + // This creates a problem when the compaction completes, as then both: + // - the source is in the compacting set + // - the result is in the live set + // This currently causes the CANONICAL set to return both source and result when early-open is disabled, + // and is otherwise worked around by opening early the last sstable in the result set (which pushes it + // in the compacting set with EARLY openReason) and the !compacting.contains(sstable) check in the + // second loop below. + // Unfortunately there does not appear to be a way to avoid this workaround. Filtering the compacting + // set through having an early-open version in live does not work because sources are fully removed from + // the live set when they are completely exhausted. + + // Add the compacting versions first because they will be the canonical versions of compaction sources. Set canonicalSSTables = new HashSet<>(); for (SSTableReader sstable : compacting) if (sstable.openReason != SSTableReader.OpenReason.EARLY) canonicalSSTables.add(sstable); - // reason for checking if compacting contains the sstable is that if compacting has an EARLY version - // of a NORMAL sstable, we still have the canonical version of that sstable in sstables. - // note that the EARLY version is equal, but not == since it is a different instance of the same sstable. + // Add anything that is not compacting, removing any compaction result where we still have the + // compaction sources. + // note that the EARLY version is equal to the original, i.e. the set itself can guarantee early-open + // versions of sstables in compacting won't be added, but we also want to remove the results. for (SSTableReader sstable : sstables) if (!compacting.contains(sstable) && sstable.openReason != SSTableReader.OpenReason.EARLY) canonicalSSTables.add(sstable); @@ -153,15 +170,10 @@ public Iterable select(SSTableSet sstableSet) } } - public Iterable getUncompacting(Iterable candidates) + + public Iterable getNoncompacting(Iterable candidates) { - return filter(candidates, new Predicate() - { - public boolean apply(SSTableReader sstable) - { - return !compacting.contains(sstable); - } - }); + return filter(candidates, (Predicate) sstable -> !compacting.contains(sstable)); } public boolean isEmpty() @@ -241,7 +253,7 @@ public static Function> selectLive(AbstractBounds< // METHODS TO CONSTRUCT FUNCTIONS FOR MODIFYING A VIEW: // return a function to un/mark the provided readers compacting in a view - static Function updateCompacting(final Set unmark, final Iterable mark) + static Function updateCompacting(final Set unmark, final Iterable mark) { if (unmark.isEmpty() && Iterables.isEmpty(mark)) return Functions.identity(); @@ -259,7 +271,7 @@ public View apply(View view) // construct a predicate to reject views that do not permit us to mark these readers compacting; // i.e. one of them is either already compacting, has been compacted, or has been replaced - static Predicate permitCompacting(final Iterable readers) + static Predicate permitCompacting(final Iterable readers) { return new Predicate() { diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java index 24d283457e99..86ac00f2fe73 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java @@ -62,8 +62,8 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right while (!accessorL.isEmptyFromOffset(left, offsetL) && !accessorR.isEmptyFromOffset(right, offsetL)) { AbstractType comparator = getComparator(i, left, accessorL, right, accessorR, offsetL, offsetR); - offsetL += getComparatorSize(i, left, accessorL, offsetL); - offsetR += getComparatorSize(i, right, accessorR, offsetR); + offsetL += getComparatorSize(left, accessorL, offsetL); + offsetR += getComparatorSize(right, accessorR, offsetR); VL value1 = accessorL.sliceWithShortLength(left, offsetL); offsetL += accessorL.sizeWithShortLength(value1); @@ -106,10 +106,9 @@ public ByteBuffer[] split(ByteBuffer bb) boolean isStatic = readIsStatic(bb, ByteBufferAccessor.instance); int offset = startingOffset(isStatic); - int i = 0; while (!ByteBufferAccessor.instance.isEmptyFromOffset(bb, offset)) { - offset += getComparatorSize(i++, bb, ByteBufferAccessor.instance, offset); + offset += getComparatorSize(bb, ByteBufferAccessor.instance, offset); ByteBuffer value = ByteBufferAccessor.instance.sliceWithShortLength(bb, offset); offset += ByteBufferAccessor.instance.sizeWithShortLength(value); l.add(value); @@ -188,7 +187,7 @@ public String getString(V input, ValueAccessor accessor) sb.append(":"); AbstractType comparator = getAndAppendComparator(i, input, accessor, sb, offset); - offset += getComparatorSize(i, input, accessor, offset); + offset += getComparatorSize(input, accessor, offset); V value = accessor.sliceWithShortLength(input, offset); offset += accessor.sizeWithShortLength(value); @@ -285,7 +284,7 @@ public void validate(V input, ValueAccessor accessor) while (!accessor.isEmptyFromOffset(input, offset)) { AbstractType comparator = validateComparator(i, input, accessor, offset); - offset += getComparatorSize(i, input, accessor, offset); + offset += getComparatorSize(input, accessor, offset); if (accessor.sizeFromOffset(input, offset) < 2) throw new MarshalException("Not enough bytes to read value size of component " + i); @@ -317,7 +316,7 @@ public TypeSerializer getSerializer() return BytesSerializer.instance; } - abstract protected int getComparatorSize(int i, V value, ValueAccessor accessor, int offset); + abstract protected int getComparatorSize(V value, ValueAccessor accessor, int offset); /** * @return the comparator for the given component. static CompositeType will consult * @param i DynamicCompositeType will read the type information from @param bb diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractGeometricType.java b/src/java/org/apache/cassandra/db/marshal/AbstractGeometricType.java new file mode 100644 index 000000000000..92ef146aa208 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/AbstractGeometricType.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.Constants; +import org.apache.cassandra.cql3.Json; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.db.marshal.geometry.GeometricType; +import org.apache.cassandra.db.marshal.geometry.OgcGeometry; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.serializers.TypeSerializer; +import org.apache.cassandra.transport.ProtocolVersion; + +public abstract class AbstractGeometricType extends AbstractType +{ + private final TypeSerializer serializer = new TypeSerializer() + { + @Override + public ByteBuffer serialize(T geometry) + { + return geoSerializer.toWellKnownBinary(geometry); + } + + @Override + public T deserialize(V value, ValueAccessor accessor) + { + // OGCGeometry does not respect the current position of the buffer, so you need to use slice() + ByteBuffer byteBuffer = accessor.toBuffer(value); + return geoSerializer.fromWellKnownBinary(byteBuffer.slice()); + } + + @Override + public void validate(V value, ValueAccessor accessor) throws MarshalException + { + ByteBuffer byteBuffer = accessor.toBuffer(value); + int pos = byteBuffer.position(); + // OGCGeometry does not respect the current position of the buffer, so you need to use slice() + geoSerializer.fromWellKnownBinary(byteBuffer.slice()).validate(); + byteBuffer.position(pos); + } + + @Override + public String toString(T geometry) + { + return geoSerializer.toWellKnownText(geometry); + } + + @Override + public Class getType() + { + return klass; + } + }; + + private final GeometricType type; + private final Class klass; + private final OgcGeometry.Serializer geoSerializer; + + public AbstractGeometricType(GeometricType type) + { + super(ComparisonType.BYTE_ORDER); + this.type = type; + this.klass = (Class) type.getGeoClass(); + this.geoSerializer = type.getSerializer(); + } + + public GeometricType getGeoType() + { + return type; + } + + @Override + public ByteBuffer fromString(String s) throws MarshalException + { + try + { + T geometry = geoSerializer.fromWellKnownText(s); + geometry.validate(); + return geoSerializer.toWellKnownBinary(geometry); + } + catch (Exception e) + { + String parentMsg = e.getMessage() != null ? " " + e.getMessage() : ""; + String msg = String.format("Unable to make %s from '%s'", getClass().getSimpleName(), s) + parentMsg; + throw new MarshalException(msg, e); + } + } + + @Override + public Term fromJSONObject(Object parsed) throws MarshalException + { + if (!(parsed instanceof String)) + { + try + { + parsed = Json.JSON_OBJECT_MAPPER.writeValueAsString(parsed); + } + catch (IOException e) + { + throw new MarshalException(e.getMessage()); + } + } + + T geometry; + try + { + geometry = geoSerializer.fromGeoJson((String) parsed); + } + catch (MarshalException e) + { + try + { + geometry = geoSerializer.fromWellKnownText((String) parsed); + } + catch (MarshalException ignored) + { + throw new MarshalException(e.getMessage()); + } + } + geometry.validate(); + return new Constants.Value(geoSerializer.toWellKnownBinary(geometry)); + } + + @Override + public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion) + { + // OGCGeometry does not respect the current position of the buffer, so you need to use slice() + return geoSerializer.toGeoJson(geoSerializer.fromWellKnownBinary(buffer.slice())); + } + + @Override + public TypeSerializer getSerializer() + { + return serializer; + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java index 19cf849dba06..4c48e7d058f8 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java @@ -39,6 +39,9 @@ import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.github.jamm.Unmetered; import static org.apache.cassandra.db.marshal.AbstractType.ComparisonType.CUSTOM; @@ -54,6 +57,8 @@ @Unmetered public abstract class AbstractType implements Comparator, AssignmentTestable { + private final static int VARIABLE_LENGTH = -1; + public final Comparator reverseComparator; public enum ComparisonType @@ -419,11 +424,28 @@ public List> getComponents() } /** - * The length of values for this type if all values are of fixed length, -1 otherwise. + * The length of values for this type if all values are of fixed length, -1 otherwise. This has an impact on + * serialization. + * + *
  • see {@link #writeValue}
  • + *
  • see {@link #read}
  • + *
  • see {@link #writtenLength}
  • + *
  • see {@link #skipValue}
  • + *
    */ public int valueLengthIfFixed() { - return -1; + return VARIABLE_LENGTH; + } + + /** + * Checks if all values are of fixed length. + * + * @return {@code true} if all values are of fixed length, {@code false} otherwise. + */ + public final boolean isValueLengthFixed() + { + return valueLengthIfFixed() != VARIABLE_LENGTH; } // This assumes that no empty values are passed @@ -520,6 +542,14 @@ public boolean referencesUserType(V name, ValueAccessor accessor) return false; } + /** + * Whether this type is or contains any UDT. + */ + public final boolean referencesUserTypes() + { + return isUDT() || subTypes().stream().anyMatch(AbstractType::referencesUserTypes); + } + /** * Returns an instance of this type with all references to the provided user type recursively replaced with its new * definition. @@ -568,6 +598,69 @@ public AssignmentTestable.TestResult testAssignment(AbstractType receiverType return AssignmentTestable.TestResult.NOT_ASSIGNABLE; } + /** + * Produce a byte-comparable representation of the given value, i.e. a sequence of bytes that compares the same way + * using lexicographical unsigned byte comparison as the original value using the type's comparator. + * + * We use a slightly stronger requirement to be able to use the types in tuples. Precisely, for any pair x, y of + * non-equal valid values of this type and any bytes b1, b2 between 0x10 and 0xEF, + * (+ stands for concatenation) + * compare(x, y) == compareLexicographicallyUnsigned(asByteComparable(x)+b1, asByteComparable(y)+b2) + * (i.e. the values compare like the original type, and an added 0x10-0xEF byte at the end does not change that) and: + * asByteComparable(x)+b1 is not a prefix of asByteComparable(y) (weakly prefix free) + * (i.e. a valid representation of a value may be a prefix of another valid representation of a value only if the + * following byte in the latter is smaller than 0x10 or larger than 0xEF). These properties are trivially true if + * the encoding compares correctly and is prefix free, but also permits a little more freedom that enables somewhat + * more efficient encoding of arbitrary-length byte-comparable blobs. + * + * Depending on the type, this method can be called for null or empty input, in which case the output is allowed to + * be null (the clustering/tuple encoding will accept and handle it). + */ + public ByteSource asComparableBytes(ValueAccessor accessor, V value, ByteComparable.Version version) + { + if (isByteOrderComparable) + { + // When a type is byte-ordered on its own, we only need to escape it, so that we can include it in + // multi-component types and make the encoding weakly-prefix-free. + return ByteSource.of(accessor, value, version); + } + else + // default is only good for byte-comparables + throw new UnsupportedOperationException(getClass().getSimpleName() + " does not implement asComparableBytes"); + } + + public final ByteSource asComparableBytes(ByteBuffer byteBuffer, ByteComparable.Version version) + { + return asComparableBytes(ByteBufferAccessor.instance, byteBuffer, version); + } + + /** + * Translates the given byte-ordered representation to the common, non-byte-ordered binary representation of a + * payload for this abstract type (the latter, common binary representation is what we mostly work with in the + * storage engine internals). If the given bytes don't correspond to the encoding of some payload value for this + * abstract type, an {@link IllegalArgumentException} may be thrown. + * + * @param accessor value accessor used to construct the value. + * @param comparableBytes A byte-ordered representation (presumably of a payload for this abstract type). + * @param version The byte-comparable version used to construct the representation. + * @return A of a payload for this abstract type, corresponding to the given byte-ordered representation, + * constructed using the supplied value accessor. + * + * @see #asComparableBytes + */ + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + if (isByteOrderComparable) + return accessor.valueOf(ByteSourceInverse.getUnescapedBytes(comparableBytes)); + else + throw new UnsupportedOperationException(getClass().getSimpleName() + " does not implement fromComparableBytes"); + } + + public final ByteBuffer fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return fromComparableBytes(ByteBufferAccessor.instance, comparableBytes, version); + } + /** * This must be overriden by subclasses if necessary so that for any * AbstractType, this == TypeParser.parse(toString()). diff --git a/src/java/org/apache/cassandra/db/marshal/BooleanType.java b/src/java/org/apache/cassandra/db/marshal/BooleanType.java index 4ef5f95b0bfc..ef8d4fcf7b85 100644 --- a/src/java/org/apache/cassandra/db/marshal/BooleanType.java +++ b/src/java/org/apache/cassandra/db/marshal/BooleanType.java @@ -26,6 +26,8 @@ import org.apache.cassandra.serializers.BooleanSerializer; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +56,26 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return v1 - v2; } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + if (accessor.isEmpty(data)) + return null; + byte b = accessor.toByte(data); + if (b != 0) + b = 1; + return ByteSource.oneByte(b); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + if (comparableBytes == null) + return accessor.empty(); + int b = comparableBytes.next(); + return b == 1 ? accessor.valueOf(true) : accessor.valueOf(false); + } + public ByteBuffer fromString(String source) throws MarshalException { diff --git a/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java b/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java index 7d13844eb0ca..b13bb1596d95 100644 --- a/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java +++ b/src/java/org/apache/cassandra/db/marshal/ByteArrayAccessor.java @@ -234,6 +234,13 @@ public UUID toUUID(byte[] value) return new UUID(getLong(value, 0), getLong(value, 8)); } + @Override + public int putByte(byte[] dst, int offset, byte value) + { + dst[offset] = value; + return TypeSizes.BYTE_SIZE; + } + @Override public int putShort(byte[] dst, int offset, short value) { diff --git a/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java b/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java index ea9bf113833b..6b9a8d8c6a25 100644 --- a/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java +++ b/src/java/org/apache/cassandra/db/marshal/ByteArrayObjectFactory.java @@ -18,13 +18,7 @@ package org.apache.cassandra.db.marshal; -import org.apache.cassandra.db.ArrayClustering; -import org.apache.cassandra.db.ArrayClusteringBound; -import org.apache.cassandra.db.ArrayClusteringBoundary; -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.ClusteringBound; -import org.apache.cassandra.db.ClusteringBoundary; -import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.*; import org.apache.cassandra.db.rows.ArrayCell; import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.CellPath; @@ -33,7 +27,7 @@ class ByteArrayObjectFactory implements ValueAccessor.ObjectFactory { - private static final Clustering EMPTY_CLUSTERING = new ArrayClustering() + private static final Clustering EMPTY_CLUSTERING = new ArrayClustering(AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY) { public String toString(TableMetadata metadata) { @@ -41,14 +35,44 @@ public String toString(TableMetadata metadata) } }; + public static final Clustering STATIC_CLUSTERING = new ArrayClustering(AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY) + { + @Override + public Kind kind() + { + return Kind.STATIC_CLUSTERING; + } + + @Override + public String toString() + { + return "STATIC"; + } + + @Override + public String toString(TableMetadata metadata) + { + return toString(); + } + }; + static final ValueAccessor.ObjectFactory instance = new ByteArrayObjectFactory(); private ByteArrayObjectFactory() {} /** The smallest start bound, i.e. the one that starts before any row. */ - private static final ArrayClusteringBound BOTTOM_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, new byte[0][]); + private static final ArrayClusteringBound BOTTOM_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, + AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY); /** The biggest end bound, i.e. the one that ends after any row. */ - private static final ArrayClusteringBound TOP_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, new byte[0][]); + private static final ArrayClusteringBound TOP_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, + AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY); + + /** The biggest start bound, i.e. the one that starts after any row. */ + private static final ArrayClusteringBound MAX_START_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.EXCL_START_BOUND, + AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY); + /** The smallest end bound, i.e. the one that end before any row. */ + private static final ArrayClusteringBound MIN_END_BOUND = new ArrayClusteringBound(ClusteringPrefix.Kind.EXCL_END_BOUND, + AbstractArrayClusteringPrefix.EMPTY_VALUES_ARRAY); public Cell cell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, byte[] value, CellPath path) { @@ -65,6 +89,11 @@ public Clustering clustering() return EMPTY_CLUSTERING; } + public Clustering staticClustering() + { + return STATIC_CLUSTERING; + } + public ClusteringBound bound(ClusteringPrefix.Kind kind, byte[]... values) { return new ArrayClusteringBound(kind, values); @@ -72,7 +101,15 @@ public ClusteringBound bound(ClusteringPrefix.Kind kind, byte[]... value public ClusteringBound bound(ClusteringPrefix.Kind kind) { - return kind.isStart() ? BOTTOM_BOUND : TOP_BOUND; + switch (kind) + { + case EXCL_END_BOUND: return MIN_END_BOUND; + case INCL_START_BOUND: return BOTTOM_BOUND; + case INCL_END_BOUND: return TOP_BOUND; + case EXCL_START_BOUND: return MAX_START_BOUND; + default: + throw new AssertionError(String.format("Unexpected kind %s for empty bound or boundary", kind)); + } } public ClusteringBoundary boundary(ClusteringPrefix.Kind kind, byte[]... values) diff --git a/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java b/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java index a0f9c1d8d004..57a13735d303 100644 --- a/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java +++ b/src/java/org/apache/cassandra/db/marshal/ByteBufferAccessor.java @@ -238,6 +238,13 @@ public UUID toUUID(ByteBuffer value) return UUIDGen.getUUID(value); } + @Override + public int putByte(ByteBuffer dst, int offset, byte value) + { + dst.put(dst.position() + offset, value); + return TypeSizes.BYTE_SIZE; + } + @Override public int putShort(ByteBuffer dst, int offset, short value) { diff --git a/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java b/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java index 00f4646341d9..4634979a1ef5 100644 --- a/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java +++ b/src/java/org/apache/cassandra/db/marshal/ByteBufferObjectFactory.java @@ -20,35 +20,27 @@ import java.nio.ByteBuffer; -import org.apache.cassandra.db.BufferClustering; -import org.apache.cassandra.db.BufferClusteringBound; -import org.apache.cassandra.db.BufferClusteringBoundary; -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.ClusteringBound; -import org.apache.cassandra.db.ClusteringBoundary; -import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.*; 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.schema.ColumnMetadata; -import org.apache.cassandra.schema.TableMetadata; class ByteBufferObjectFactory implements ValueAccessor.ObjectFactory { - /** Empty clustering for tables having no clustering columns. */ - private static final Clustering EMPTY_CLUSTERING = new BufferClustering() - { - @Override - public String toString(TableMetadata metadata) - { - return "EMPTY"; - } - }; - /** The smallest start bound, i.e. the one that starts before any row. */ - private static final BufferClusteringBound BOTTOM_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, new ByteBuffer[0]); + private static final BufferClusteringBound BOTTOM_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_START_BOUND, + AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY); /** The biggest end bound, i.e. the one that ends after any row. */ - private static final BufferClusteringBound TOP_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, new ByteBuffer[0]); + private static final BufferClusteringBound TOP_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, + AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY); + + /** The biggest start bound, i.e. the one that starts after any row. */ + private static final BufferClusteringBound MAX_START_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.EXCL_START_BOUND, + AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY); + /** The smallest end bound, i.e. the one that end before any row. */ + private static final BufferClusteringBound MIN_END_BOUND = new BufferClusteringBound(ClusteringPrefix.Kind.EXCL_END_BOUND, + AbstractBufferClusteringPrefix.EMPTY_VALUES_ARRAY); static final ValueAccessor.ObjectFactory instance = new ByteBufferObjectFactory(); @@ -66,7 +58,11 @@ public Clustering clustering(ByteBuffer... values) public Clustering clustering() { - return EMPTY_CLUSTERING; + return Clustering.EMPTY; + } + + public Clustering staticClustering() { + return Clustering.STATIC_CLUSTERING; } public ClusteringBound bound(ClusteringPrefix.Kind kind, ByteBuffer... values) @@ -76,7 +72,15 @@ public ClusteringBound bound(ClusteringPrefix.Kind kind, ByteBuffer. public ClusteringBound bound(ClusteringPrefix.Kind kind) { - return kind.isStart() ? BOTTOM_BOUND : TOP_BOUND; + switch (kind) + { + case EXCL_END_BOUND: return MIN_END_BOUND; + case INCL_START_BOUND: return BOTTOM_BOUND; + case INCL_END_BOUND: return TOP_BOUND; + case EXCL_START_BOUND: return MAX_START_BOUND; + default: + throw new AssertionError(String.format("Unexpected kind %s for empty bound or boundary", kind)); + } } public ClusteringBoundary boundary(ClusteringPrefix.Kind kind, ByteBuffer... values) diff --git a/src/java/org/apache/cassandra/db/marshal/ByteType.java b/src/java/org/apache/cassandra/db/marshal/ByteType.java index f94f4bb01cc5..a910fbba11a1 100644 --- a/src/java/org/apache/cassandra/db/marshal/ByteType.java +++ b/src/java/org/apache/cassandra/db/marshal/ByteType.java @@ -27,6 +27,10 @@ import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class ByteType extends NumberType { @@ -42,6 +46,19 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return accessorL.getByte(left, 0) - accessorR.getByte(right, 0); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + // This type does not allow non-present values, but we do just to avoid future complexity. + return ByteSource.optionalSignedFixedLengthNumber(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 1); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java index bf5e914a9d9e..e348c9a1c5fc 100644 --- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java +++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java @@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -31,6 +32,9 @@ import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import static com.google.common.collect.Iterables.any; import static com.google.common.collect.Iterables.transform; @@ -165,6 +169,84 @@ protected AbstractType getAndAppendComparator(int i, V value, ValueAccess return types.get(i); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + if (data == null || accessor.isEmpty(data)) + return null; + + ByteSource[] srcs = new ByteSource[types.size() * 2 + 1]; + int length = accessor.size(data); + + // statics go first + boolean isStatic = readIsStaticInternal(data, accessor); + int offset = startingOffsetInternal(isStatic); + srcs[0] = isStatic ? null : ByteSource.EMPTY; + + int i = 0; + byte lastEoc = 0; + while (offset < length) + { + // Only the end-of-component byte of the last component of this composite can be non-zero, so the + // component before can't have a non-zero end-of-component byte. + assert lastEoc == 0 : lastEoc; + + int componentLength = accessor.getUnsignedShort(data, offset); + offset += 2; + srcs[i * 2 + 1] = types.get(i).asComparableBytes(accessor, accessor.slice(data, offset, componentLength), version); + offset += componentLength; + lastEoc = accessor.getByte(data, offset); + offset += 1; + srcs[i * 2 + 2] = ByteSource.oneByte(lastEoc & 0xFF ^ 0x80); // end-of-component also takes part in comparison as signed byte + ++i; + } + if (i * 2 + 1 < srcs.length) + srcs = Arrays.copyOfRange(srcs, 0, i * 2 + 1); + + return ByteSource.withTerminator(version == Version.LEGACY ? ByteSource.END_OF_STREAM : ByteSource.TERMINATOR, + srcs); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, Version version) + { + // For ByteComparable.Version.LEGACY the terminator byte is ByteSource.END_OF_STREAM. The latter means that it's + // indistinguishable from the END_OF_STREAM byte that gets returned _after_ the terminator byte has already + // been consumed, when the composite is part of a multi-component sequence. So if in such a scenario we consume + // the ByteSource.END_OF_STREAM terminator here, this will result in actually consuming the multi-component + // sequence separator after it and jumping directly into the bytes of the next component, when we try to + // consume the (already consumed) separator. + // Instead of trying to find a way around the situation, we can just take advantage of the fact that we don't + // need to decode from Version.LEGACY, assume that we never do that, and assert it here. + assert version != Version.LEGACY; + + if (comparableBytes == null) + return accessor.empty(); + + int separator = comparableBytes.next(); + boolean isStatic = ByteSourceInverse.nextComponentNull(separator); + int i = 0; + V[] buffers = accessor.createArray(types.size()); + byte lastEoc = 0; + + while ((separator = comparableBytes.next()) != ByteSource.TERMINATOR && i < types.size()) + { + // Only the end-of-component byte of the last component of this composite can be non-zero, so the + // component before can't have a non-zero end-of-component byte. + assert lastEoc == 0 : lastEoc; + + // Get the next type and decode its payload. + AbstractType type = types.get(i); + V decoded = type.fromComparableBytes(accessor, + ByteSourceInverse.nextComponentSource(comparableBytes, separator), + version); + buffers[i++] = decoded; + + lastEoc = ByteSourceInverse.getSignedByte(ByteSourceInverse.nextComponentSource(comparableBytes)); + } + return build(accessor, isStatic, Arrays.copyOf(buffers, i), lastEoc); + } + protected ParsedComparator parseComparator(int i, String part) { return new StaticParsedComparator(types.get(i), part); @@ -177,7 +259,7 @@ protected AbstractType validateComparator(int i, V value, ValueAccessor int getComparatorSize(int i, V value, ValueAccessor accessor, int offset) + protected int getComparatorSize(V value, ValueAccessor accessor, int offset) { return 0; } @@ -389,4 +471,27 @@ public static V build(ValueAccessor accessor, boolean isStatic, V... valu out.flip(); return accessor.valueOf(out); } + + @VisibleForTesting + public static V build(ValueAccessor accessor, boolean isStatic, V[] values, byte lastEoc) + { + int totalLength = isStatic ? 2 : 0; + for (V v : values) + totalLength += 2 + accessor.size(v) + 1; + + ByteBuffer out = ByteBuffer.allocate(totalLength); + + if (isStatic) + out.putShort((short)STATIC_MARKER); + + for (int i = 0; i < values.length; ++i) + { + V v = values[i]; + ByteBufferUtil.writeShortLength(out, accessor.size(v)); + accessor.write(v, out); + out.put(i != values.length - 1 ? (byte) 0 : lastEoc); + } + out.flip(); + return accessor.valueOf(out); + } } diff --git a/src/java/org/apache/cassandra/db/marshal/DateRangeType.java b/src/java/org/apache/cassandra/db/marshal/DateRangeType.java new file mode 100644 index 000000000000..dc1136463f6d --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/DateRangeType.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal; + + +import java.nio.ByteBuffer; + +import org.apache.cassandra.cql3.Constants; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.db.marshal.datetime.DateRange; +import org.apache.cassandra.db.marshal.datetime.DateRangeUtil; +import org.apache.cassandra.serializers.DateRangeSerializer; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.serializers.TypeSerializer; +import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.ByteBufferUtil; + + +/** + * Date range C* type with lower and upper bounds represented as timestamps with a millisecond precision. + */ +public class DateRangeType extends AbstractType +{ + public static final DateRangeType instance = new DateRangeType(); + + private DateRangeType() + { + super(ComparisonType.BYTE_ORDER); + } + + @Override + public ByteBuffer fromString(String source) throws MarshalException + { + if (source.isEmpty()) + { + return ByteBufferUtil.EMPTY_BYTE_BUFFER; + } + try + { + DateRange dateRange = DateRangeUtil.parseDateRange(source); + return decompose(dateRange); + } + catch (Exception e) + { + throw new MarshalException(String.format("Could not parse date range: %s %s", source, e.getMessage()), e); + } + } + + @Override + public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion) + { + DateRange dateRange = this.getSerializer().deserialize(buffer); + return '"' + dateRange.formatToSolrString() + '"'; + } + + @Override + public Term fromJSONObject(Object parsed) throws MarshalException + { + if (parsed instanceof String) + { + return new Constants.Value(fromString((String) parsed)); + } + throw new MarshalException(String.format( + "Expected a string representation of a date range value, but got a %s: %s", + parsed.getClass().getSimpleName(), parsed)); + } + + @Override + public boolean isEmptyValueMeaningless() + { + return true; + } + + @Override + public TypeSerializer getSerializer() + { + return DateRangeSerializer.instance; + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/DateType.java b/src/java/org/apache/cassandra/db/marshal/DateType.java index 473cedf40795..595106d3d184 100644 --- a/src/java/org/apache/cassandra/db/marshal/DateType.java +++ b/src/java/org/apache/cassandra/db/marshal/DateType.java @@ -31,6 +31,9 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; /** * This is the old version of TimestampType, but has been replaced as it wasn't comparing pre-epoch timestamps @@ -50,6 +53,19 @@ public boolean isEmptyValueMeaningless() return true; } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + // While BYTE_ORDER would still work for this type, making use of the fixed length is more efficient. + return ByteSource.optionalFixedLength(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalFixedLength(accessor, comparableBytes, 8); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java index 5740fdcc0fcb..44782890ef0b 100644 --- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java +++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java @@ -24,6 +24,8 @@ import java.nio.ByteBuffer; import java.util.Objects; +import com.google.common.primitives.Ints; + import org.apache.cassandra.cql3.CQL3Type; import org.apache.cassandra.cql3.Constants; import org.apache.cassandra.cql3.Term; @@ -32,6 +34,8 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; public class DecimalType extends NumberType { @@ -41,6 +45,16 @@ public class DecimalType extends NumberType private static final int MAX_SCALE = 1000; private static final MathContext MAX_PRECISION = new MathContext(10000); + // Constants or escaping values needed to encode/decode variable-length floating point numbers (decimals) in our + // custom byte-ordered encoding scheme. + private static final int POSITIVE_DECIMAL_HEADER_MASK = 0x80; + private static final int NEGATIVE_DECIMAL_HEADER_MASK = 0x00; + private static final int DECIMAL_EXPONENT_LENGTH_HEADER_MASK = 0x40; + private static final byte DECIMAL_LAST_BYTE = (byte) 0x00; + private static final BigInteger HUNDRED = BigInteger.valueOf(100); + + private static final ByteBuffer ZERO_BUFFER = instance.decompose(BigDecimal.ZERO); + DecimalType() {super(ComparisonType.CUSTOM);} // singleton public boolean isEmptyValueMeaningless() @@ -59,6 +73,196 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return compareComposed(left, accessorL, right, accessorR, this); } + /** + * Constructs a byte-comparable representation. + * This is rather difficult and involves reconstructing the decimal. + * + * To compare, we need a normalized value, i.e. one with a sign, exponent and (0,1) mantissa. To avoid + * loss of precision, both exponent and mantissa need to be base-100. We can't get this directly off the serialized + * bytes, as they have base-10 scale and base-256 unscaled part. + * + * We store: + * - sign bit inverted * 0x80 + 0x40 + signed exponent length, where exponent is negated if value is negative + * - zero or more exponent bytes (as given by length) + * - 0x80 + first pair of decimal digits, negative is value is negative, rounded to -inf + * - zero or more 0x80 + pair of decimal digits, always positive + * - trailing 0x00 + * Zero is special-cased as 0x80. + * + * Because the trailing 00 cannot be produced from a pair of decimal digits (positive or not), no value can be + * a prefix of another. + * + * Encoding examples: + * 1.1 as c1 = 0x80 (positive number) + 0x40 + (positive exponent) 0x01 (exp length 1) + * 01 = exponent 1 (100^1) + * 81 = 0x80 + 01 (0.01) + * 8a = 0x80 + 10 (....10) 0.0110e2 + * 00 + * -1 as 3f = 0x00 (negative number) + 0x40 - (negative exponent) 0x01 (exp length 1) + * ff = exponent -1. negative number, thus 100^1 + * 7f = 0x80 - 01 (-0.01) -0.01e2 + * 00 + * -99.9 as 3f = 0x00 (negative number) + 0x40 - (negative exponent) 0x01 (exp length 1) + * ff = exponent -1. negative number, thus 100^1 + * 1c = 0x80 - 100 (-1.00) + * 8a = 0x80 + 10 (+....10) -0.999e2 + * 00 + * + */ + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + BigDecimal value = compose(data, accessor); + if (value == null) + return null; + if (value.compareTo(BigDecimal.ZERO) == 0) // Note: 0.equals(0.0) returns false! + return ByteSource.oneByte(POSITIVE_DECIMAL_HEADER_MASK); + + long scale = (((long) value.scale()) - value.precision()) & ~1; + boolean negative = value.signum() < 0; + // Make a base-100 exponent (this will always fit in an int). + int exponent = Math.toIntExact(-scale >> 1); + // Flip the exponent sign for negative numbers, so that ones with larger magnitudes are propely treated as smaller. + final int modulatedExponent = negative ? -exponent : exponent; + // We should never have scale > Integer.MAX_VALUE, as we're always subtracting the non-negative precision of + // the encoded BigDecimal, and furthermore we're rounding to negative infinity. + assert scale <= Integer.MAX_VALUE; + // However, we may end up overflowing on the negative side. + if (scale < Integer.MIN_VALUE) + { + // As scaleByPowerOfTen needs an int scale, do the scaling in two steps. + int mv = Integer.MIN_VALUE; + value = value.scaleByPowerOfTen(mv); + scale -= mv; + } + final BigDecimal mantissa = value.scaleByPowerOfTen(Ints.checkedCast(scale)).stripTrailingZeros(); + // We now have a smaller-than-one signed mantissa, and a signed and modulated base-100 exponent. + assert mantissa.abs().compareTo(BigDecimal.ONE) < 0; + + return new ByteSource() + { + // Start with up to 5 bytes for sign + exponent. + int exponentBytesLeft = 5; + BigDecimal current = mantissa; + + @Override + public int next() + { + if (exponentBytesLeft > 0) + { + --exponentBytesLeft; + if (exponentBytesLeft == 4) + { + // Skip leading zero bytes in the modulatedExponent. + exponentBytesLeft -= Integer.numberOfLeadingZeros(Math.abs(modulatedExponent)) / 8; + // Now prepare the leading byte which includes the sign of the number plus the sign and length of the modulatedExponent. + int explen = DECIMAL_EXPONENT_LENGTH_HEADER_MASK + (modulatedExponent < 0 ? -exponentBytesLeft : exponentBytesLeft); + return explen + (negative ? NEGATIVE_DECIMAL_HEADER_MASK : POSITIVE_DECIMAL_HEADER_MASK); + } + else + return (modulatedExponent >> (exponentBytesLeft * 8)) & 0xFF; + } + else if (current == null) + { + return END_OF_STREAM; + } + else if (current.compareTo(BigDecimal.ZERO) == 0) + { + current = null; + return 0x00; + } + else + { + BigDecimal v = current.scaleByPowerOfTen(2); + BigDecimal floor = v.setScale(0, BigDecimal.ROUND_FLOOR); + current = v.subtract(floor); + return floor.byteValueExact() + 0x80; + } + } + }; + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + if (comparableBytes == null) + return accessor.empty(); + + int headerBits = comparableBytes.next(); + if (headerBits == POSITIVE_DECIMAL_HEADER_MASK) + return accessor.valueOf(ZERO_BUFFER); + + // I. Extract the exponent. + // The sign of the decimal, and the sign and the length (in bytes) of the decimal exponent, are all encoded in + // the first byte. + // Get the sign of the decimal... + boolean isNegative = headerBits < POSITIVE_DECIMAL_HEADER_MASK; + headerBits -= isNegative ? NEGATIVE_DECIMAL_HEADER_MASK : POSITIVE_DECIMAL_HEADER_MASK; + headerBits -= DECIMAL_EXPONENT_LENGTH_HEADER_MASK; + // Get the sign and the length of the exponent (the latter is encoded as its negative if the sign of the + // exponent is negative)... + boolean isExponentNegative = headerBits < 0; + headerBits = isExponentNegative ? -headerBits : headerBits; + // Now consume the exponent bytes. If the exponent is negative and uses less than 4 bytes, the remaining bytes + // should be padded with 1s, in order for the constructed int to contain the correct (negative) exponent value. + // So, if the exponent is negative, we can just start with all bits set to 1 (i.e. we can start with -1). + int exponent = isExponentNegative ? -1 : 0; + for (int i = 0; i < headerBits; ++i) + exponent = (exponent << 8) | comparableBytes.next(); + // The encoded exponent also contains the decimal sign, in order to correctly compare exponents in case of + // negative decimals (e.g. x * 10^y > x * 10^z if x < 0 && y < z). After the decimal sign is "removed", what's + // left is a base-100 exponent following BigDecimal's convention for the exponent sign. + exponent = isNegative ? -exponent : exponent; + + // II. Extract the mantissa as a BigInteger value. It was encoded as a BigDecimal value between 0 and 1, in + // order to be used for comparison (after the sign of the decimal and the sign and the value of the exponent), + // but when decoding we don't need that property on the transient mantissa value. + BigInteger mantissa = BigInteger.ZERO; + int curr = comparableBytes.next(); + while (curr != DECIMAL_LAST_BYTE) + { + // The mantissa value is constructed by a standard positional notation value calculation. + // The value of the next digit is the next most-significant mantissa byte as an unsigned integer, + // offset by a predetermined value (in this case, 0x80)... + int currModified = curr - 0x80; + // ...multiply the current value by the base (in this case, 100)... + mantissa = mantissa.multiply(HUNDRED); + // ...then add the next digit to the modified current value... + mantissa = mantissa.add(BigInteger.valueOf(currModified)); + // ...and finally, adjust the base-100, BigDecimal format exponent accordingly. + --exponent; + curr = comparableBytes.next(); + } + + // III. Construct the final BigDecimal value, by combining the mantissa and the exponent, guarding against + // underflow or overflow when exponents are close to their boundary values. + long base10NonBigDecimalFormatExp = 2L * exponent; + // When expressing a sufficiently big decimal, BigDecimal's internal scale value will be negative with very + // big absolute value. To compute the encoded exponent, this internal scale has the number of digits of the + // unscaled value subtracted from it, after which it's divided by 2, rounding down to negative infinity + // (before accounting for the decimal sign). When decoding, this exponent is converted to a base-10 exponent in + // non-BigDecimal format, which means that it can very well overflow Integer.MAX_VALUE. + // For example, see how new BigDecimal(BigInteger.TEN, Integer.MIN_VALUE) is encoded and decoded. + if (base10NonBigDecimalFormatExp > Integer.MAX_VALUE) + { + // If the base-10 exponent will result in an overflow, some of its powers of 10 need to be absorbed by the + // mantissa. How much exactly? As little as needed, in order to avoid complex BigInteger operations, which + // means exactly as much as to have a scale of -Integer.MAX_VALUE. + int exponentReduction = (int) (base10NonBigDecimalFormatExp - Integer.MAX_VALUE); + mantissa = mantissa.multiply(BigInteger.TEN.pow(exponentReduction)); + base10NonBigDecimalFormatExp = Integer.MAX_VALUE; + } + assert base10NonBigDecimalFormatExp >= Integer.MIN_VALUE && base10NonBigDecimalFormatExp <= Integer.MAX_VALUE; + // Here we negate the exponent, as we are not using BigDecimal.scaleByPowerOfTen, where a positive number means + // "multiplying by a positive power of 10", but to BigDecimal's internal scale representation, where a positive + // number means "dividing by a positive power of 10". + byte[] mantissaBytes = mantissa.toByteArray(); + V resultBuf = accessor.allocate(4 + mantissaBytes.length); + accessor.putInt(resultBuf, 0, (int) -base10NonBigDecimalFormatExp); + accessor.copyByteArrayTo(mantissaBytes, 0, resultBuf, 4, mantissaBytes.length); + return resultBuf; + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/DoubleType.java b/src/java/org/apache/cassandra/db/marshal/DoubleType.java index 570d420a75bb..56ae0131b3a9 100644 --- a/src/java/org/apache/cassandra/db/marshal/DoubleType.java +++ b/src/java/org/apache/cassandra/db/marshal/DoubleType.java @@ -27,6 +27,9 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class DoubleType extends NumberType { @@ -50,6 +53,18 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return compareComposed(left, accessorL, right, accessorR, this); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + return ByteSource.optionalSignedFixedLengthFloat(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalSignedFixedLengthFloat(accessor, comparableBytes, 8); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java index 5df36009956e..a0f909c7fc96 100644 --- a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java +++ b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java @@ -19,9 +19,16 @@ import java.nio.ByteBuffer; import java.nio.charset.CharacterCodingException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +41,9 @@ import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import static com.google.common.collect.Iterables.any; @@ -60,7 +70,11 @@ public class DynamicCompositeType extends AbstractCompositeType { private static final Logger logger = LoggerFactory.getLogger(DynamicCompositeType.class); + private static final ByteSource[] EMPTY_BYTE_SOURCE_ARRAY = new ByteSource[0]; + private static final String REVERSED_TYPE = ReversedType.class.getSimpleName(); + private final Map> aliases; + private final Map, Byte> inverseMapping; // interning instances private static final ConcurrentHashMap>, DynamicCompositeType> instances = new ConcurrentHashMap<>(); @@ -81,6 +95,9 @@ public static DynamicCompositeType getInstance(Map> aliase private DynamicCompositeType(Map> aliases) { this.aliases = aliases; + this.inverseMapping = new HashMap<>(); + for (Map.Entry> en : aliases.entrySet()) + this.inverseMapping.put(en.getValue(), en.getKey()); } protected boolean readIsStatic(V value, ValueAccessor accessor) @@ -94,7 +111,7 @@ protected int startingOffset(boolean isStatic) return 0; } - protected int getComparatorSize(int i, V value, ValueAccessor accessor, int offset) + protected int getComparatorSize(V value, ValueAccessor accessor, int offset) { int header = accessor.getShort(value, offset); if ((header & 0x8000) == 0) @@ -114,7 +131,6 @@ private AbstractType getComparator(V value, ValueAccessor accessor, in int header = accessor.getShort(value, offset); if ((header & 0x8000) == 0) { - String name = accessor.toString(accessor.slice(value, offset + 2, header)); return TypeParser.parse(name); } @@ -197,6 +213,194 @@ protected AbstractType getAndAppendComparator(int i, V value, ValueAccess } } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + List srcs = new ArrayList<>(); + int length = accessor.size(data); + + // statics go first + boolean isStatic = readIsStatic(data, accessor); + int offset = startingOffset(isStatic); + srcs.add(isStatic ? null : ByteSource.EMPTY); + + byte lastEoc = 0; + while (offset < length) + { + // Only the end-of-component byte of the last component of this composite can be non-zero, so the + // component before can't have a non-zero end-of-component byte. + assert lastEoc == 0 : lastEoc; + + AbstractType comp = getComparator(data, accessor, offset); + offset += getComparatorSize(data, accessor, offset); + // The comparable bytes for the component need to ensure comparisons consistent with + // AbstractCompositeType.compareCustom(ByteBuffer, ByteBuffer) and + // DynamicCompositeType.getComparator(int, ByteBuffer, ByteBuffer): + if (version == Version.LEGACY || !(comp instanceof ReversedType)) + { + // ...most often that means just adding the short name of the type, followed by the full name of the type. + srcs.add(ByteSource.of(comp.getClass().getSimpleName(), version)); + srcs.add(ByteSource.of(comp.getClass().getName(), version)); + } + else + { + // ...however some times the component uses a complex type (currently the only supported complex type + // is ReversedType - we can't have elements that are of MapType, CompositeType, TupleType, etc.)... + ReversedType reversedComp = (ReversedType) comp; + // ...in this case, we need to add the short name of ReversedType before the short name of the base + // type, to ensure consistency with DynamicCompositeType.getComparator(int, ByteBuffer, ByteBuffer). + srcs.add(ByteSource.of(REVERSED_TYPE, version)); + srcs.add(ByteSource.of(reversedComp.baseType.getClass().getSimpleName(), version)); + srcs.add(ByteSource.of(reversedComp.baseType.getClass().getName(), version)); + } + // Only then the payload of the component gets encoded. + int componentLength = accessor.getUnsignedShort(data, offset); + offset += 2; + srcs.add(comp.asComparableBytes(accessor, accessor.slice(data, offset, componentLength), version)); + offset += componentLength; + // The end-of-component byte also takes part in the comparison, and therefore needs to be encoded. + lastEoc = accessor.getByte(data, offset); + offset += 1; + srcs.add(ByteSource.oneByte(version == Version.LEGACY ? lastEoc : lastEoc & 0xFF ^ 0x80)); + } + + return ByteSource.withTerminator(version == Version.LEGACY ? ByteSource.END_OF_STREAM : ByteSource.TERMINATOR, + srcs.toArray(EMPTY_BYTE_SOURCE_ARRAY)); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, Version version) + { + // For ByteComparable.Version.LEGACY the terminator byte is ByteSource.END_OF_STREAM. Just like with + // CompositeType, this means that in multi-component sequences the terminator may be transformed to a regular + // component separator, but unlike CompositeType (where we have the expected number of types/components), + // this can make the end of the whole dynamic composite type indistinguishable from the end of a component + // somewhere in the middle of the dynamic composite type. Because of that, DynamicCompositeType elements + // cannot always be safely decoded using that encoding version. + // Even more so than with CompositeType, we just take advantage of the fact that we don't need to decode from + // Version.LEGACY, assume that we never do that, and assert it here. + assert version != Version.LEGACY; + + if (comparableBytes == null) + return accessor.empty(); + + // The first byte is the isStatic flag which we don't need but must consume to continue past it. + comparableBytes.next(); + + List> types = new ArrayList<>(); + List values = new ArrayList<>(); + byte lastEoc = 0; + + for (int separator = comparableBytes.next(); separator != ByteSource.TERMINATOR; separator = comparableBytes.next()) + { + // Solely the end-of-component byte of the last component of this composite can be non-zero. + assert lastEoc == 0 : lastEoc; + + boolean isReversed = false; + // Decode the next type's simple class name that is encoded before its fully qualified class name (in order + // for comparisons to work correctly). + String simpleClassName = ByteSourceInverse.getString(ByteSourceInverse.nextComponentSource(comparableBytes, separator)); + if (REVERSED_TYPE.equals(simpleClassName)) + { + // Special-handle if the type is reversed (and decode the actual base type simple class name). + isReversed = true; + simpleClassName = ByteSourceInverse.getString(ByteSourceInverse.nextComponentSource(comparableBytes)); + } + + // Decode the type's fully qualified class name and parse the actual type from it. + String fullClassName = ByteSourceInverse.getString(ByteSourceInverse.nextComponentSource(comparableBytes)); + assert fullClassName.endsWith(simpleClassName); + AbstractType type = isReversed ? ReversedType.getInstance(TypeParser.parse(fullClassName)) + : TypeParser.parse(fullClassName); + assert type != null; + types.add(type); + + // Decode the payload from this type. + V value = type.fromComparableBytes(accessor, ByteSourceInverse.nextComponentSource(comparableBytes), version); + values.add(value); + + // Also decode the corresponding end-of-component byte - the last one we decode will be taken into + // account when we deserialize the decoded data into an object. + lastEoc = ByteSourceInverse.getSignedByte(ByteSourceInverse.nextComponentSource(comparableBytes)); + } + return build(accessor, types, inverseMapping, values, lastEoc); + } + + public static ByteBuffer build(List types, List values) + { + return build(ByteBufferAccessor.instance, + Lists.transform(types, TypeParser::parse), + Collections.emptyMap(), + values, + (byte) 0); + } + + @VisibleForTesting + public static V build(ValueAccessor accessor, + List> types, + Map, Byte> inverseMapping, + List values, + byte lastEoc) + { + assert types.size() == values.size(); + + int numComponents = types.size(); + // Compute the total number of bytes that we'll need to store the types and their payloads. + int totalLength = 0; + for (int i = 0; i < numComponents; ++i) + { + AbstractType type = types.get(i); + Byte alias = inverseMapping.get(type); + int typeNameLength = alias == null ? type.toString().getBytes(StandardCharsets.UTF_8).length : 0; + // The type data will be stored by means of the type's fully qualified name, not by aliasing, so: + // 1. The type data header should be the fully qualified name length in bytes. + // 2. The length should be small enough so that it fits in 15 bits (2 bytes with the first bit zero). + assert typeNameLength <= 0x7FFF; + int valueLength = accessor.size(values.get(i)); + // The value length should also expect its first bit to be 0, as the length should be stored as a signed + // 2-byte value (short). + assert valueLength <= 0x7FFF; + totalLength += 2 + typeNameLength + 2 + valueLength + 1; + } + + V result = accessor.allocate(totalLength); + int offset = 0; + for (int i = 0; i < numComponents; ++i) + { + AbstractType type = types.get(i); + Byte alias = inverseMapping.get(type); + if (alias == null) + { + // Write the type data (2-byte length header + the fully qualified type name in UTF-8). + byte[] typeNameBytes = type.toString().getBytes(StandardCharsets.UTF_8); + accessor.putShort(result, + offset, + (short) typeNameBytes.length); // this should work fine also if length >= 32768 + offset += 2; + accessor.copyByteArrayTo(typeNameBytes, 0, result, offset, typeNameBytes.length); + offset += typeNameBytes.length; + } + else + { + accessor.putShort(result, offset, (short) (alias | 0x8000)); + offset += 2; + } + + // Write the type payload data (2-byte length header + the payload). + V value = values.get(i); + int bytesToCopy = accessor.size(value); + accessor.putShort(result, offset, (short) bytesToCopy); + offset += 2; + accessor.copyTo(value, 0, result, accessor, offset, bytesToCopy); + offset += bytesToCopy; + + // Write the end-of-component byte. + accessor.putByte(result, offset, i != numComponents - 1 ? (byte) 0 : lastEoc); + offset += 1; + } + return result; + } + protected ParsedComparator parseComparator(int i, String part) { return new DynamicParsedComparator(part); diff --git a/src/java/org/apache/cassandra/db/marshal/EmptyType.java b/src/java/org/apache/cassandra/db/marshal/EmptyType.java index 357b6e85ad15..dcc57b7c4a75 100644 --- a/src/java/org/apache/cassandra/db/marshal/EmptyType.java +++ b/src/java/org/apache/cassandra/db/marshal/EmptyType.java @@ -33,6 +33,8 @@ import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.NoSpamLogger; /** @@ -68,6 +70,18 @@ private static NonEmptyWriteBehavior parseNonEmptyWriteBehavior() private EmptyType() {super(ComparisonType.CUSTOM);} // singleton + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + return null; + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return accessor.empty(); + } + public int compareCustom(VL left, ValueAccessor accessorL, VR right, ValueAccessor accessorR) { return 0; diff --git a/src/java/org/apache/cassandra/db/marshal/FloatType.java b/src/java/org/apache/cassandra/db/marshal/FloatType.java index 35abee0f98ed..2adb127d4194 100644 --- a/src/java/org/apache/cassandra/db/marshal/FloatType.java +++ b/src/java/org/apache/cassandra/db/marshal/FloatType.java @@ -27,6 +27,9 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class FloatType extends NumberType @@ -51,6 +54,18 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return compareComposed(left, accessorL, right, accessorR, this); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + return ByteSource.optionalSignedFixedLengthFloat(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalSignedFixedLengthFloat(accessor, comparableBytes, 4); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/GeometryCodec.java b/src/java/org/apache/cassandra/db/marshal/GeometryCodec.java new file mode 100644 index 000000000000..62c8804235e9 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/GeometryCodec.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal; + +import java.nio.ByteBuffer; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.ProtocolVersion; +import com.datastax.driver.core.TypeCodec; +import com.datastax.driver.core.exceptions.InvalidTypeException; +import org.apache.cassandra.db.marshal.geometry.LineString; +import org.apache.cassandra.db.marshal.geometry.OgcGeometry; +import org.apache.cassandra.db.marshal.geometry.Point; +import org.apache.cassandra.db.marshal.geometry.Polygon; + +public class GeometryCodec extends TypeCodec +{ + public static final TypeCodec pointCodec = new GeometryCodec<>(PointType.instance); + public static final TypeCodec lineStringCodec = new GeometryCodec<>(LineStringType.instance); + public static final TypeCodec polygonCodec = new GeometryCodec<>(PolygonType.instance); + + private final OgcGeometry.Serializer serializer; + + public GeometryCodec(AbstractGeometricType type) + { + super(DataType.custom(type.getClass().getName()), (Class) type.getGeoType().getGeoClass()); + this.serializer = (OgcGeometry.Serializer) type.getGeoType().getSerializer(); + } + + @Override + public T deserialize(ByteBuffer bb, ProtocolVersion protocolVersion) throws InvalidTypeException + { + return bb == null || bb.remaining() == 0 ? null : serializer.fromWellKnownBinary(bb); + } + + @Override + public ByteBuffer serialize(T geometry, ProtocolVersion protocolVersion) throws InvalidTypeException + { + return geometry == null ? null : geometry.asWellKnownBinary(); + } + + @Override + public T parse(String s) throws InvalidTypeException + { + if (s == null || s.isEmpty() || s.equalsIgnoreCase("NULL")) + return null; + return serializer.fromWellKnownText(s); + } + + @Override + public String format(T geometry) throws InvalidTypeException + { + return geometry == null ? "NULL" : geometry.asWellKnownText(); + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/Int32Type.java b/src/java/org/apache/cassandra/db/marshal/Int32Type.java index 98f4c83cf64c..6dee26e22423 100644 --- a/src/java/org/apache/cassandra/db/marshal/Int32Type.java +++ b/src/java/org/apache/cassandra/db/marshal/Int32Type.java @@ -28,6 +28,9 @@ import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class Int32Type extends NumberType { @@ -55,6 +58,18 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return ValueAccessor.compare(left, accessorL, right, accessorR); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + return ByteSource.optionalSignedFixedLengthNumber(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 4); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/IntegerType.java b/src/java/org/apache/cassandra/db/marshal/IntegerType.java index 4c913d50afee..30a4fbea503d 100644 --- a/src/java/org/apache/cassandra/db/marshal/IntegerType.java +++ b/src/java/org/apache/cassandra/db/marshal/IntegerType.java @@ -30,11 +30,21 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; public final class IntegerType extends NumberType { public static final IntegerType instance = new IntegerType(); + // Constants or escaping values needed to encode/decode variable-length integers in our custom byte-ordered + // encoding scheme. + private static final int POSITIVE_VARINT_HEADER = 0x80; + private static final int NEGATIVE_VARINT_LENGTH_HEADER = 0x00; + private static final int POSITIVE_VARINT_LENGTH_HEADER = 0xFF; + private static final byte BIG_INTEGER_NEGATIVE_LEADING_ZERO = (byte) 0xFF; + private static final byte BIG_INTEGER_POSITIVE_LEADING_ZERO = (byte) 0x00; + private static int findMostSignificantByte(V value, ValueAccessor accessor) { int len = accessor.size(value) - 1; @@ -131,6 +141,143 @@ public static int compareIntegers(VL lhs, ValueAccessor accessorL, return 0; } + /** + * Constructs a byte-comparable representation of the number. + * We represent it as + * + * where a length_byte is: + * - 0x80 + (length - 1) for positive numbers (so that longer length sorts bigger) + * - 0x7F - (length - 1) for negative numbers (so that longer length sorts smaller) + * + * Because we include the sign in the length byte: + * - unlike fixed-length ints, we don't need to sign-invert the first significant byte, + * - unlike BigInteger, we don't need to include 0x00 prefix for positive integers whose first byte is >= 0x80 + * or 0xFF prefix for negative integers whose first byte is < 0x80. + * + * The representations are prefix-free, because representations of different length always have length bytes that + * differ. + * + * Examples: + * 0 as 8000 + * 1 as 8001 + * 127 as 807F + * 255 as 80FF + * 2^32-1 as 837FFFFFFF + * 2^32 as 8380000000 + * 2^33 as 840100000000 + */ + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + int p = 0; + final int limit = accessor.size(data); + if (p == limit) + return null; + + // skip any leading sign-only byte(s) + final byte signbyte = accessor.getByte(data, p); + if (signbyte == BIG_INTEGER_NEGATIVE_LEADING_ZERO || signbyte == BIG_INTEGER_POSITIVE_LEADING_ZERO) + { + while (p + 1 < limit) + { + if (accessor.getByte(data, ++p) != signbyte) + break; + } + } + + final int startpos = p; + + return new ByteSource() + { + int pos = startpos; + int sizeToReport = limit - startpos; + boolean sizeReported = false; + + public int next() + { + if (!sizeReported) + { + if (sizeToReport >= 128) + { + sizeToReport -= 128; + return signbyte >= 0 + ? POSITIVE_VARINT_LENGTH_HEADER + : NEGATIVE_VARINT_LENGTH_HEADER; + } + else + { + sizeReported = true; + return signbyte >= 0 + ? POSITIVE_VARINT_HEADER + (sizeToReport - 1) + : POSITIVE_VARINT_HEADER - sizeToReport; + } + } + + if (pos == limit) + return END_OF_STREAM; + + return accessor.getByte(data, pos++) & 0xFF; + } + }; + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + if (comparableBytes == null) + return accessor.empty(); + + int valueBytes; + byte signedZero; + // Consume the first byte to determine whether the encoded number is positive and + // start iterating through the length header bytes and collecting the number of value bytes. + int curr = comparableBytes.next(); + if (curr >= POSITIVE_VARINT_HEADER) // positive number + { + valueBytes = curr - POSITIVE_VARINT_HEADER + 1; + while (curr == POSITIVE_VARINT_LENGTH_HEADER) + { + curr = comparableBytes.next(); + valueBytes += curr - POSITIVE_VARINT_HEADER + 1; + } + signedZero = 0; + } + else // negative number + { + valueBytes = POSITIVE_VARINT_HEADER - curr; + while (curr == NEGATIVE_VARINT_LENGTH_HEADER) + { + curr = comparableBytes.next(); + valueBytes += POSITIVE_VARINT_HEADER - curr; + } + signedZero = -1; + } + + int writtenBytes = 0; + V buf; + // Add "leading zero" if needed (i.e. in case the leading byte of a positive number corresponds to a negative + // value, or in case the leading byte of a negative number corresponds to a non-negative value). + // Size the array containing all the value bytes accordingly. + curr = comparableBytes.next(); + if ((curr & 0x80) != (signedZero & 0x80)) + { + ++valueBytes; + buf = accessor.allocate(valueBytes); + accessor.putByte(buf, writtenBytes++, signedZero); + } + else + buf = accessor.allocate(valueBytes); + // Don't forget to add the first consumed value byte after determining whether leading zero should be added + // and sizing the value bytes array. + accessor.putByte(buf, writtenBytes++, (byte) curr); + + // Consume exactly the number of expected value bytes. + while (writtenBytes < valueBytes) + accessor.putByte(buf, writtenBytes++, (byte) comparableBytes.next()); + + return buf; + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java index 6dd41616f04d..81ec9d9a566e 100644 --- a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java +++ b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java @@ -26,6 +26,9 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.UUIDSerializer; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class LexicalUUIDType extends AbstractType { @@ -48,6 +51,46 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return accessorL.toUUID(left).compareTo(accessorR.toUUID(right)); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + if (data == null || accessor.isEmpty(data)) + return null; + + // fixed-length (hence prefix-free) representation, but + // we have to sign-flip the highest bytes of the two longs + return new ByteSource() + { + int bufpos = 0; + + public int next() + { + if (bufpos >= accessor.size(data)) + return END_OF_STREAM; + int v = accessor.getByte(data, bufpos) & 0xFF; + if (bufpos == 0 || bufpos == 8) + v ^= 0x80; + ++bufpos; + return v; + } + }; + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + // Optional-style encoding of empty values as null sources + if (comparableBytes == null) + return accessor.empty(); + + long hiBits = ByteSourceInverse.getSignedLong(comparableBytes); + long loBits = ByteSourceInverse.getSignedLong(comparableBytes); + + // Lexical UUIDs are stored as just two signed longs. The decoding of these longs flips their sign bit back, so + // they can directly be used for constructing the original UUID. + return UUIDType.makeUuidBytes(accessor, hiBits, loBits); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/compaction/ActiveCompactionsTracker.java b/src/java/org/apache/cassandra/db/marshal/LineStringType.java similarity index 64% rename from src/java/org/apache/cassandra/db/compaction/ActiveCompactionsTracker.java rename to src/java/org/apache/cassandra/db/marshal/LineStringType.java index c1bbbd8e67bf..07400eab1fc9 100644 --- a/src/java/org/apache/cassandra/db/compaction/ActiveCompactionsTracker.java +++ b/src/java/org/apache/cassandra/db/marshal/LineStringType.java @@ -16,19 +16,17 @@ * limitations under the License. */ -package org.apache.cassandra.db.compaction; +package org.apache.cassandra.db.marshal; -public interface ActiveCompactionsTracker +import org.apache.cassandra.db.marshal.geometry.GeometricType; +import org.apache.cassandra.db.marshal.geometry.LineString; + +public class LineStringType extends AbstractGeometricType { - public void beginCompaction(CompactionInfo.Holder ci); - public void finishCompaction(CompactionInfo.Holder ci); + public static final LineStringType instance = new LineStringType(); - public static final ActiveCompactionsTracker NOOP = new ActiveCompactionsTracker() + public LineStringType() { - public void beginCompaction(CompactionInfo.Holder ci) - {} - - public void finishCompaction(CompactionInfo.Holder ci) - {} - }; + super(GeometricType.LINESTRING); + } } diff --git a/src/java/org/apache/cassandra/db/marshal/ListType.java b/src/java/org/apache/cassandra/db/marshal/ListType.java index cc6393751bcc..8a48870dce63 100644 --- a/src/java/org/apache/cassandra/db/marshal/ListType.java +++ b/src/java/org/apache/cassandra/db/marshal/ListType.java @@ -18,7 +18,11 @@ package org.apache.cassandra.db.marshal; import java.nio.ByteBuffer; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import org.apache.cassandra.cql3.Json; @@ -32,6 +36,9 @@ import org.apache.cassandra.serializers.ListSerializer; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class ListType extends CollectionType> { @@ -195,6 +202,63 @@ static int compareListOrSet(AbstractType elementsComparator, VL left return sizeL == sizeR ? 0 : (sizeL < sizeR ? -1 : 1); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + return asComparableBytesListOrSet(getElementsType(), accessor, data, version); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, Version version) + { + return fromComparableBytesListOrSet(accessor, comparableBytes, version, getElementsType()); + } + + static ByteSource asComparableBytesListOrSet(AbstractType elementsComparator, + ValueAccessor accessor, + V data, + Version version) + { + if (accessor.isEmpty(data)) + return null; + + int offset = 0; + int size = CollectionSerializer.readCollectionSize(data, accessor, ProtocolVersion.V3); + offset += CollectionSerializer.sizeOfCollectionSize(size, ProtocolVersion.V3); + ByteSource[] srcs = new ByteSource[size]; + for (int i = 0; i < size; ++i) + { + V v = CollectionSerializer.readValue(data, accessor, offset, ProtocolVersion.V3); + offset += CollectionSerializer.sizeOfValue(v, accessor, ProtocolVersion.V3); + srcs[i] = elementsComparator.asComparableBytes(accessor, v, version); + } + return ByteSource.withTerminator(version == Version.LEGACY ? 0x00 : ByteSource.TERMINATOR, srcs); + } + + static V fromComparableBytesListOrSet(ValueAccessor accessor, + ByteSource.Peekable comparableBytes, + Version version, + AbstractType elementType) + { + if (comparableBytes == null) + return accessor.empty(); + + List buffers = new ArrayList<>(); + int terminator = version == Version.LEGACY + ? 0x00 + : ByteSource.TERMINATOR; + int separator = comparableBytes.next(); + while (separator != terminator) + { + if (!ByteSourceInverse.nextComponentNull(separator)) + buffers.add(elementType.fromComparableBytes(accessor, comparableBytes, version)); + else + buffers.add(null); + separator = comparableBytes.next(); + } + return CollectionSerializer.pack(buffers, accessor, buffers.size(), ProtocolVersion.V3); + } + @Override public String toString(boolean ignoreFreezing) { @@ -245,7 +309,7 @@ public static String setOrListToJsonString(ByteBuffer buffer, AbstractType eleme { ByteBuffer value = buffer.duplicate(); StringBuilder sb = new StringBuilder("["); - int size = CollectionSerializer.readCollectionSize(value, protocolVersion); + int size = CollectionSerializer.readCollectionSize(value, ByteBufferAccessor.instance, protocolVersion); int offset = CollectionSerializer.sizeOfCollectionSize(size, protocolVersion); for (int i = 0; i < size; i++) { diff --git a/src/java/org/apache/cassandra/db/marshal/LongType.java b/src/java/org/apache/cassandra/db/marshal/LongType.java index ad539f70de70..9e0335733745 100644 --- a/src/java/org/apache/cassandra/db/marshal/LongType.java +++ b/src/java/org/apache/cassandra/db/marshal/LongType.java @@ -28,6 +28,9 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class LongType extends NumberType { @@ -57,6 +60,18 @@ public static int compareLongs(VL left, ValueAccessor accessorL, VR return ValueAccessor.compare(left, accessorL, right, accessorR); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + return ByteSource.optionalSignedFixedLengthNumber(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 8); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java index 9473e2913618..5ae5ed936168 100644 --- a/src/java/org/apache/cassandra/db/marshal/MapType.java +++ b/src/java/org/apache/cassandra/db/marshal/MapType.java @@ -28,9 +28,12 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.serializers.CollectionSerializer; -import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.MapSerializer; +import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.apache.cassandra.utils.Pair; public class MapType extends CollectionType> @@ -218,6 +221,72 @@ public static int compareMaps(AbstractType keysComparator, AbstractT return sizeL == sizeR ? 0 : (sizeL < sizeR ? -1 : 1); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + return asComparableBytesMap(getKeysType(), getValuesType(), accessor, data, version); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, Version version) + { + return fromComparableBytesMap(accessor, comparableBytes, version, getKeysType(), getValuesType()); + } + + static ByteSource asComparableBytesMap(AbstractType keysComparator, + AbstractType valuesComparator, + ValueAccessor accessor, + V data, + Version version) + { + if (accessor.isEmpty(data)) + return null; + + ProtocolVersion protocolVersion = ProtocolVersion.V3; + int offset = 0; + int size = CollectionSerializer.readCollectionSize(data, accessor, protocolVersion); + offset += CollectionSerializer.sizeOfCollectionSize(size, protocolVersion); + ByteSource[] srcs = new ByteSource[size * 2]; + for (int i = 0; i < size; ++i) + { + V k = CollectionSerializer.readValue(data, accessor, offset, protocolVersion); + offset += CollectionSerializer.sizeOfValue(k, accessor, protocolVersion); + srcs[i * 2 + 0] = keysComparator.asComparableBytes(accessor, k, version); + V v = CollectionSerializer.readValue(data, accessor, offset, protocolVersion); + offset += CollectionSerializer.sizeOfValue(v, accessor, protocolVersion); + srcs[i * 2 + 1] = valuesComparator.asComparableBytes(accessor, v, version); + } + return ByteSource.withTerminator(version == Version.LEGACY ? 0x00 : ByteSource.TERMINATOR, srcs); + } + + static V fromComparableBytesMap(ValueAccessor accessor, + ByteSource.Peekable comparableBytes, + Version version, + AbstractType keysComparator, + AbstractType valuesComparator) + { + if (comparableBytes == null) + return accessor.empty(); + + List buffers = new ArrayList<>(); + int terminator = version == Version.LEGACY + ? 0x00 + : ByteSource.TERMINATOR; + int separator = comparableBytes.next(); + while (separator != terminator) + { + buffers.add(ByteSourceInverse.nextComponentNull(separator) + ? null + : keysComparator.fromComparableBytes(accessor, comparableBytes, version)); + separator = comparableBytes.next(); + buffers.add(ByteSourceInverse.nextComponentNull(separator) + ? null + : valuesComparator.fromComparableBytes(accessor, comparableBytes, version)); + separator = comparableBytes.next(); + } + return CollectionSerializer.pack(buffers, accessor,buffers.size() / 2, ProtocolVersion.V3); + } + @Override public MapSerializer getSerializer() { @@ -286,7 +355,7 @@ public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion) { ByteBuffer value = buffer.duplicate(); StringBuilder sb = new StringBuilder("{"); - int size = CollectionSerializer.readCollectionSize(value, protocolVersion); + int size = CollectionSerializer.readCollectionSize(value, ByteBufferAccessor.instance, protocolVersion); int offset = CollectionSerializer.sizeOfCollectionSize(size, protocolVersion); for (int i = 0; i < size; i++) { diff --git a/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java b/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java index 89241b416bb4..91fe3e31059c 100644 --- a/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java +++ b/src/java/org/apache/cassandra/db/marshal/PartitionerDefinedOrder.java @@ -22,11 +22,15 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.FBUtilities; /** for sorting columns representing row keys in the row ordering as determined by a partitioner. @@ -93,6 +97,33 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return PartitionPosition.ForKey.get(accessorL.toBuffer(left), partitioner).compareTo(PartitionPosition.ForKey.get(accessorR.toBuffer(right), partitioner)); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + // Partitioners work with ByteBuffers only. + ByteBuffer buf = ByteBufferAccessor.instance.convert(data, accessor); + if (version != Version.LEGACY) + { + // For ByteComparable.Version.OSS41 and above we encode an empty key with a null byte source. This + // way we avoid the need to special-handle a sentinel value when we decode the byte source for such a key + // (e.g. for ByteComparable.Version.Legacy we use the minimum key bound of the partitioner's minimum token as + // a sentinel value, and that results in the need to go twice through the byte source that is being + // decoded). + return buf.hasRemaining() ? partitioner.decorateKey(buf).asComparableBytes(version) : null; + } + return PartitionPosition.ForKey.get(buf, partitioner).asComparableBytes(version); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + assert version != Version.LEGACY; + if (comparableBytes == null) + return accessor.empty(); + byte[] keyBytes = DecoratedKey.keyFromByteComparable(v -> comparableBytes, version, partitioner); + return accessor.valueOf(keyBytes); + } + @Override public void validate(ByteBuffer bytes) throws MarshalException { diff --git a/src/java/org/apache/cassandra/db/marshal/PointType.java b/src/java/org/apache/cassandra/db/marshal/PointType.java new file mode 100644 index 000000000000..487f73553172 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/PointType.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal; + +import org.apache.cassandra.db.marshal.geometry.GeometricType; +import org.apache.cassandra.db.marshal.geometry.Point; + +public class PointType extends AbstractGeometricType +{ + public static final PointType instance = new PointType(); + + public PointType() + { + super(GeometricType.POINT); + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/PolygonType.java b/src/java/org/apache/cassandra/db/marshal/PolygonType.java new file mode 100644 index 000000000000..a0bb2b186d4c --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/PolygonType.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal; + +import org.apache.cassandra.db.marshal.geometry.GeometricType; +import org.apache.cassandra.db.marshal.geometry.Polygon; + +public class PolygonType extends AbstractGeometricType +{ + public static final PolygonType instance = new PolygonType(); + + public PolygonType() + { + super(GeometricType.POLYGON); + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java index 8a4b58dca297..21f3340b0950 100644 --- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java +++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java @@ -28,6 +28,8 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; public class ReversedType extends AbstractType { @@ -63,6 +65,32 @@ public boolean isEmptyValueMeaningless() return baseType.isEmptyValueMeaningless(); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + ByteSource src = baseType.asComparableBytes(accessor, data, version); + if (src == null) // Note: this will only compare correctly if used within a sequence + return null; + // Invert all bytes. + // The comparison requirements for the original type ensure that this encoding will compare correctly with + // respect to the reversed comparator function (and, specifically, prefixes of escaped byte-ordered types will + // compare as larger). Additionally, the weak prefix-freedom requirement ensures this encoding will also be + // weakly prefix-free. + return () -> + { + int v = src.next(); + if (v == ByteSource.END_OF_STREAM) + return v; + return v ^ 0xFF; + }; + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return baseType.fromComparableBytes(accessor, ReversedPeekableByteSource.of(comparableBytes), version); + } + public int compareCustom(VL left, ValueAccessor accessorL, VR right, ValueAccessor accessorR) { return baseType.compare(right, accessorR, left, accessorL); @@ -162,4 +190,38 @@ public String toString() { return getClass().getName() + "(" + baseType + ")"; } + + private static final class ReversedPeekableByteSource extends ByteSource.Peekable + { + private final ByteSource.Peekable original; + + static ByteSource.Peekable of(ByteSource.Peekable original) + { + return original != null ? new ReversedPeekableByteSource(original) : null; + } + + private ReversedPeekableByteSource(ByteSource.Peekable original) + { + super(null); + this.original = original; + } + + @Override + public int next() + { + int v = original.next(); + if (v != END_OF_STREAM) + return v ^ 0xFF; + return END_OF_STREAM; + } + + @Override + public int peek() + { + int v = original.peek(); + if (v != END_OF_STREAM) + return v ^ 0xFF; + return END_OF_STREAM; + } + } } diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java index e5bdadab25f8..2b83afbbc05f 100644 --- a/src/java/org/apache/cassandra/db/marshal/SetType.java +++ b/src/java/org/apache/cassandra/db/marshal/SetType.java @@ -30,6 +30,8 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.SetSerializer; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; public class SetType extends CollectionType> { @@ -157,6 +159,18 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return ListType.compareListOrSet(elements, left, accessorL, right, accessorR); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + return ListType.asComparableBytesListOrSet(getElementsType(), accessor, data, version); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ListType.fromComparableBytesListOrSet(accessor, comparableBytes, version, getElementsType()); + } + public SetSerializer getSerializer() { return serializer; diff --git a/src/java/org/apache/cassandra/db/marshal/ShortType.java b/src/java/org/apache/cassandra/db/marshal/ShortType.java index 03dcf5d31446..013fa959497a 100644 --- a/src/java/org/apache/cassandra/db/marshal/ShortType.java +++ b/src/java/org/apache/cassandra/db/marshal/ShortType.java @@ -28,6 +28,9 @@ import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; public class ShortType extends NumberType { @@ -46,6 +49,19 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return ValueAccessor.compare(left, accessorL, right, accessorR); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + // This type does not allow non-present values, but we do just to avoid future complexity. + return ByteSource.optionalSignedFixedLengthNumber(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 2); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java index f883ccdc1c54..a0de2c20892e 100644 --- a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java +++ b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java @@ -23,12 +23,15 @@ import org.apache.cassandra.cql3.Constants; import org.apache.cassandra.cql3.Duration; import org.apache.cassandra.cql3.Term; -import org.apache.cassandra.cql3.statements.RequestValidations; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.SimpleDateSerializer; import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; @@ -38,6 +41,20 @@ public class SimpleDateType extends TemporalType SimpleDateType() {super(ComparisonType.BYTE_ORDER);} // singleton + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + // While BYTE_ORDER would still work for this type, making use of the fixed length is more efficient. + // This type does not allow non-present values, but we do just to avoid future complexity. + return ByteSource.optionalFixedLength(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalFixedLength(accessor, comparableBytes, 4); + } + public ByteBuffer fromString(String source) throws MarshalException { return ByteBufferUtil.bytes(SimpleDateSerializer.dateStringToDays(source)); diff --git a/src/java/org/apache/cassandra/db/marshal/TimeType.java b/src/java/org/apache/cassandra/db/marshal/TimeType.java index be20ba7a526e..f029b8bb94a8 100644 --- a/src/java/org/apache/cassandra/db/marshal/TimeType.java +++ b/src/java/org/apache/cassandra/db/marshal/TimeType.java @@ -19,7 +19,6 @@ import java.nio.ByteBuffer; import java.time.LocalTime; -import java.time.ZoneId; import java.time.ZoneOffset; import org.apache.cassandra.cql3.Constants; @@ -29,6 +28,10 @@ import org.apache.cassandra.serializers.TypeSerializer; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteComparable.Version; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; /** * Nanosecond resolution time values @@ -43,6 +46,20 @@ public ByteBuffer fromString(String source) throws MarshalException return decompose(TimeSerializer.timeStringToLong(source)); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, Version version) + { + // While BYTE_ORDER would still work for this type, making use of the fixed length is more efficient. + // This type does not allow non-present values, but we do just to avoid future complexity. + return ByteSource.optionalFixedLength(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalFixedLength(accessor, comparableBytes, 8); + } + @Override public boolean isValueCompatibleWithInternal(AbstractType otherType) { diff --git a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java index 6cf137596ccf..67f2d0bda9cc 100644 --- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java +++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java @@ -21,10 +21,12 @@ import java.util.UUID; import org.apache.cassandra.cql3.CQL3Type; -import org.apache.cassandra.cql3.ColumnSpecification; import org.apache.cassandra.cql3.Constants; import org.apache.cassandra.cql3.Term; import org.apache.cassandra.serializers.TypeSerializer; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.apache.cassandra.utils.UUIDGen; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.TimeUUIDSerializer; @@ -57,12 +59,12 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right long msb1 = accessorL.getLong(left, 0); long msb2 = accessorR.getLong(right, 0); + verifyVersion(msb1); + verifyVersion(msb2); + msb1 = reorderTimestampBytes(msb1); msb2 = reorderTimestampBytes(msb2); - assert (msb1 & topbyte(0xf0L)) == topbyte(0x10L); - assert (msb2 & topbyte(0xf0L)) == topbyte(0x10L); - int c = Long.compare(msb1, msb2); if (c != 0) return c; @@ -74,6 +76,40 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return Long.compare(lsb1, lsb2); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + if (accessor.isEmpty(data)) + return null; + + long hiBits = accessor.getLong(data, 0); + verifyVersion(hiBits); + ByteBuffer swizzled = ByteBuffer.allocate(16); + swizzled.putLong(0, TimeUUIDType.reorderTimestampBytes(hiBits)); + swizzled.putLong(8, accessor.getLong(data, 8) ^ 0x8080808080808080L); + + return ByteSource.fixedLength(swizzled); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + // Optional-style encoding of empty values as null sources + if (comparableBytes == null) + return accessor.empty(); + + // The non-lexical UUID bits are stored as an unsigned fixed-length 128-bit integer. + long hiBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8); + long loBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8); + + hiBits = reorderBackTimestampBytes(hiBits); + verifyVersion(hiBits); + // In addition, TimeUUIDType also touches the low bits of the UUID (see CASSANDRA-8730 and DB-1758). + loBits ^= 0x8080808080808080L; + + return UUIDType.makeUuidBytes(accessor, hiBits, loBits); + } + // takes as input 8 signed bytes in native machine order // returns the first byte unchanged, and the following 7 bytes converted to an unsigned representation // which is the same as a 2's complement long in native format @@ -82,16 +118,30 @@ private static long signedBytesToNativeLong(long signedBytes) return signedBytes ^ 0x0080808080808080L; } - private static long topbyte(long topbyte) + private void verifyVersion(long hiBits) { - return topbyte << 56; + long version = (hiBits >>> 12) & 0xf; + if (version != 1) + throw new MarshalException(String.format("Invalid UUID version %d for timeuuid", + version)); } protected static long reorderTimestampBytes(long input) { - return (input << 48) - | ((input << 16) & 0xFFFF00000000L) - | (input >>> 32); + return (input << 48) + | ((input << 16) & 0xFFFF00000000L) + | (input >>> 32); + } + + protected static long reorderBackTimestampBytes(long input) + { + // In a time-based UUID the high bits are significantly more shuffled than in other UUIDs - if [X] represents a + // 16-bit tuple, [1][2][3][4] should become [3][4][2][1]. + // See the UUID Javadoc (and more specifically the high bits layout of a Leach-Salz UUID) to understand the + // reasoning behind this bit twiddling in the first place (in the context of comparisons). + return (input << 32) + | ((input >>> 16) & 0xFFFF0000L) + | (input >>> 48); } public ByteBuffer fromString(String source) throws MarshalException diff --git a/src/java/org/apache/cassandra/db/marshal/TimestampType.java b/src/java/org/apache/cassandra/db/marshal/TimestampType.java index 0dac6b0394d2..5bca7b1f56db 100644 --- a/src/java/org/apache/cassandra/db/marshal/TimestampType.java +++ b/src/java/org/apache/cassandra/db/marshal/TimestampType.java @@ -23,7 +23,6 @@ import org.apache.cassandra.cql3.Constants; import org.apache.cassandra.cql3.Duration; import org.apache.cassandra.cql3.Term; -import org.apache.cassandra.cql3.statements.RequestValidations; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,6 +32,9 @@ import org.apache.cassandra.serializers.TimestampSerializer; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; @@ -61,6 +63,18 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return LongType.compareLongs(left, accessorL, right, accessorR); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + return ByteSource.optionalSignedFixedLengthNumber(accessor, data); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return ByteSourceInverse.getOptionalSignedFixedLength(accessor, comparableBytes, 8); + } + public ByteBuffer fromString(String source) throws MarshalException { // Return an empty ByteBuffer for an empty string. diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java index 83fbb25d548a..c8a8dbda7b8f 100644 --- a/src/java/org/apache/cassandra/db/marshal/TupleType.java +++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java @@ -30,11 +30,12 @@ import org.apache.cassandra.cql3.*; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.serializers.*; import org.apache.cassandra.transport.ProtocolVersion; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import static com.google.common.collect.Iterables.any; import static com.google.common.collect.Iterables.transform; @@ -63,7 +64,7 @@ public TupleType(List> types) this(types, true); } - protected TupleType(List> types, boolean freezeInner) + public TupleType(List> types, boolean freezeInner) { super(ComparisonType.CUSTOM); @@ -200,39 +201,87 @@ private boolean allRemainingComponentsAreNull(T v, ValueAccessor accessor return true; } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version version) + { + if (accessor.isEmpty(data)) + return null; + + V[] bufs = split(accessor, data); // this may be shorter than types.size -- other srcs remain null in that case + ByteSource[] srcs = new ByteSource[types.size()]; + for (int i = 0; i < bufs.length; ++i) + srcs[i] = bufs[i] != null ? types.get(i).asComparableBytes(accessor, bufs[i], version) : null; + // We always have a fixed number of sources, with the trailing ones possibly being nulls. + // This can only result in a prefix if the last type in the tuple allows prefixes. Since that type is required + // to be weakly prefix-free, so is the tuple. + return ByteSource.withTerminator(ByteSource.END_OF_STREAM, srcs); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + if (comparableBytes == null) + return accessor.empty(); + + V[] componentBuffers = accessor.createArray(types.size()); + for (int i = 0; i < types.size(); ++i) + { + AbstractType componentType = types.get(i); + ByteSource.Peekable component = ByteSourceInverse.nextComponentSource(comparableBytes); + if (component != null) + componentBuffers[i] = componentType.fromComparableBytes(accessor, component, version); + else + componentBuffers[i] = null; + } + return buildValue(accessor, componentBuffers); + } + /** * Split a tuple value into its component values. */ - public ByteBuffer[] split(ByteBuffer value) + public V[] split(ValueAccessor accessor, V value) { - ByteBuffer[] components = new ByteBuffer[size()]; - ByteBuffer input = value.duplicate(); + V[] components = accessor.createArray(size()); + int length = accessor.size(value); + int position = 0; for (int i = 0; i < size(); i++) { - if (!input.hasRemaining()) + if (position == length) return Arrays.copyOfRange(components, 0, i); - int size = input.getInt(); - - if (input.remaining() < size) + if (position + 4 > length) throw new MarshalException(String.format("Not enough bytes to read %dth component", i)); + int size = accessor.getInt(value, position); + position += 4; + // size < 0 means null value - components[i] = size < 0 ? null : ByteBufferUtil.readBytes(input, size); + if (size >= 0) + { + if (position + size > length) + throw new MarshalException(String.format("Not enough bytes to read %dth component", i)); + + components[i] = accessor.slice(value, position, size); + position += size; + } + else + components[i] = null; } // error out if we got more values in the tuple/UDT than we expected - if (input.hasRemaining()) + if (position < length) { - throw new InvalidRequestException(String.format( - "Expected %s %s for %s column, but got more", - size(), size() == 1 ? "value" : "values", this.asCQL3Type())); + throw new MarshalException(String.format("Expected %s %s for %s column, but got more", + size(), + size() == 1 ? "value" : "values", + this.asCQL3Type())); } return components; } - public static V buildValue(ValueAccessor accessor, V[] components) + @SafeVarargs + public static V buildValue(ValueAccessor accessor, V... components) { int totalLength = 0; for (V component : components) @@ -256,7 +305,7 @@ public static V buildValue(ValueAccessor accessor, V[] components) return result; } - public static ByteBuffer buildValue(ByteBuffer[] components) + public static ByteBuffer buildValue(ByteBuffer... components) { return buildValue(ByteBufferAccessor.instance, components); } diff --git a/src/java/org/apache/cassandra/db/marshal/UUIDType.java b/src/java/org/apache/cassandra/db/marshal/UUIDType.java index 55ce59dae798..7978ca469cda 100644 --- a/src/java/org/apache/cassandra/db/marshal/UUIDType.java +++ b/src/java/org/apache/cassandra/db/marshal/UUIDType.java @@ -30,6 +30,9 @@ import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.serializers.UUIDSerializer; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.apache.cassandra.utils.UUIDGen; /** @@ -96,9 +99,71 @@ public int compareCustom(VL left, ValueAccessor accessorL, VR right return c; } + // Amusingly (or not so much), although UUIDType freely takes time UUIDs (UUIDs with version 1), it compares + // them differently than TimeUUIDType. This is evident in the least significant bytes comparison (the code + // below for UUIDType), where UUIDType treats them as unsigned bytes, while TimeUUIDType compares the bytes + // signed. See CASSANDRA-8730 for details around this discrepancy. return UnsignedLongs.compare(accessorL.getLong(left, 8), accessorR.getLong(right, 8)); } + @Override + public ByteSource asComparableBytes(ValueAccessor accessor, V data, ByteComparable.Version v) + { + if (accessor.isEmpty(data)) + return null; + + long msb = accessor.getLong(data, 0); + long version = ((msb >>> 12) & 0xf); + ByteBuffer swizzled = ByteBuffer.allocate(16); + + if (version == 1) + swizzled.putLong(0, TimeUUIDType.reorderTimestampBytes(msb)); + else + swizzled.putLong(0, (version << 60) | ((msb >>> 4) & 0x0FFFFFFFFFFFF000L) | (msb & 0xFFFL)); + + swizzled.putLong(8, accessor.getLong(data, 8)); + + // fixed-length thus prefix-free + return ByteSource.fixedLength(swizzled); + } + + @Override + public V fromComparableBytes(ValueAccessor accessor, ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + // Optional-style encoding of empty values as null sources + if (comparableBytes == null) + return accessor.empty(); + + // The UUID bits are stored as an unsigned fixed-length 128-bit integer. + long hiBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8); + long loBits = ByteSourceInverse.getUnsignedFixedLengthAsLong(comparableBytes, 8); + + long version1 = hiBits >>> 60 & 0xF; + if (version1 == 1) + { + // If the version bits are set to 1, this is a time-based UUID, and its high bits are significantly more + // shuffled than in other UUIDs. Revert the shuffle. + hiBits = TimeUUIDType.reorderBackTimestampBytes(hiBits); + } + else + { + // For non-time UUIDs, the only thing that's needed is to put the version bits back where they were originally. + hiBits = hiBits << 4 & 0xFFFFFFFFFFFF0000L + | version1 << 12 + | hiBits & 0x0000000000000FFFL; + } + + return makeUuidBytes(accessor, hiBits, loBits); + } + + static V makeUuidBytes(ValueAccessor accessor, long high, long low) + { + V buffer = accessor.allocate(16); + accessor.putLong(buffer, 0, high); + accessor.putLong(buffer, 8, low); + return buffer; + } + @Override public boolean isValueCompatibleWithInternal(AbstractType otherType) { diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java index 29afad9583d4..24c05e255967 100644 --- a/src/java/org/apache/cassandra/db/marshal/UserType.java +++ b/src/java/org/apache/cassandra/db/marshal/UserType.java @@ -258,7 +258,7 @@ public Term fromJSONObject(Object parsed) throws MarshalException @Override public String toJSONString(ByteBuffer buffer, ProtocolVersion protocolVersion) { - ByteBuffer[] buffers = split(buffer); + ByteBuffer[] buffers = split(ByteBufferAccessor.instance, buffer); StringBuilder sb = new StringBuilder("{"); for (int i = 0; i < types.size(); i++) { diff --git a/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java b/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java index 10532ff304bb..5fe0b3006538 100644 --- a/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java +++ b/src/java/org/apache/cassandra/db/marshal/ValueAccessor.java @@ -66,6 +66,7 @@ public interface ObjectFactory Cell cell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, V value, CellPath path); Clustering clustering(V... values); Clustering clustering(); + Clustering staticClustering(); ClusteringBound bound(ClusteringPrefix.Kind kind, V... values); ClusteringBound bound(ClusteringPrefix.Kind kind); ClusteringBoundary boundary(ClusteringPrefix.Kind kind, V... values); @@ -103,7 +104,6 @@ default ClusteringBoundary exclusiveCloseInclusiveOpen(boolean reversed, V[] { return boundary(reversed ? INCL_END_EXCL_START_BOUNDARY : EXCL_END_INCL_START_BOUNDARY, boundValues); } - } /** * @return the size of the given value @@ -322,6 +322,12 @@ default boolean getBoolean(V value, int offset) /** returns a UUID from offset 0 */ UUID toUUID(V value); + /** + * writes the byte value {@param value} to {@param dst} at offset {@param offset} + * @return the number of bytes written to {@param value} + */ + int putByte(V dst, int offset, byte value); + /** * writes the short value {@param value} to {@param dst} at offset {@param offset} * @return the number of bytes written to {@param value} diff --git a/src/java/org/apache/cassandra/db/marshal/datetime/DateRange.java b/src/java/org/apache/cassandra/db/marshal/datetime/DateRange.java new file mode 100644 index 000000000000..8fe5ba1df77c --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/datetime/DateRange.java @@ -0,0 +1,403 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal.datetime; + +import java.time.Instant; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.util.Locale; + +import com.google.common.base.MoreObjects; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.builder.EqualsBuilder; + +import org.apache.cassandra.db.marshal.DateRangeType; + +import static java.time.temporal.ChronoField.DAY_OF_MONTH; +import static java.time.temporal.ChronoField.HOUR_OF_DAY; +import static java.time.temporal.ChronoField.MILLI_OF_SECOND; +import static java.time.temporal.ChronoField.MINUTE_OF_HOUR; +import static java.time.temporal.ChronoField.MONTH_OF_YEAR; +import static java.time.temporal.ChronoField.SECOND_OF_MINUTE; + +/** + * Domain object of type {@link DateRangeType}. Lower and upper bounds are inclusive. Value type. + */ +public class DateRange +{ + private final DateRangeBound lowerBound; + private final DateRangeBound upperBound; + + public DateRange(DateRangeBound lowerBound) + { + Preconditions.checkArgument(lowerBound != null); + this.lowerBound = lowerBound; + this.upperBound = null; + } + + public DateRange(DateRangeBound lowerBound, DateRangeBound upperBound) + { + Preconditions.checkArgument(lowerBound != null); + Preconditions.checkArgument(upperBound != null); + Preconditions.checkArgument(upperBound.isAfter(lowerBound), "Wrong order: " + lowerBound + " TO " + upperBound); + this.lowerBound = lowerBound; + this.upperBound = upperBound; + } + + private DateRange(DateRangeBuilder builder) + { + this.lowerBound = builder.lowerBound; + this.upperBound = builder.upperBound; + } + + public DateRangeBound getLowerBound() + { + return lowerBound; + } + + public DateRangeBound getUpperBound() + { + return upperBound; + } + + public boolean isUpperBoundDefined() + { + return upperBound != null; + } + + public String formatToSolrString() + { + if (isUpperBoundDefined()) + { + return String.format("[%s TO %s]", lowerBound, upperBound); + } + else + { + return lowerBound.toString(); + } + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("lowerBound", lowerBound) + .add("precision", lowerBound.getPrecision()) + .add("upperBound", upperBound) + .add("precision", upperBound != null ? upperBound.getPrecision() : "null") + .toString(); + } + + @Override + public boolean equals(Object obj) + { + if (obj == null || obj.getClass() != getClass()) + { + return false; + } + if (obj == this) + { + return true; + } + + DateRange rhs = (DateRange) obj; + return new EqualsBuilder() + .append(lowerBound, rhs.lowerBound) + .append(upperBound, rhs.upperBound) + .isEquals(); + } + + @Override + public int hashCode() + { + return Objects.hashCode(lowerBound, upperBound); + } + + public static class DateRangeBound + { + public static final DateRangeBound UNBOUNDED = new DateRangeBound(); + + private final ZonedDateTime timestamp; + private final Precision precision; + + private DateRangeBound(ZonedDateTime timestamp, Precision precision) + { + Preconditions.checkArgument(timestamp != null); + Preconditions.checkArgument(precision != null); + this.timestamp = timestamp; + this.precision = precision; + } + + private DateRangeBound() + { + this.timestamp = null; + this.precision = null; + } + + public static DateRangeBound lowerBound(Instant timestamp, Precision precision) + { + return lowerBound(ZonedDateTime.ofInstant(timestamp, ZoneOffset.UTC), precision); + } + + public static DateRangeBound lowerBound(ZonedDateTime timestamp, Precision precision) + { + ZonedDateTime roundedLowerBound = DateRangeUtil.roundLowerBoundTimestampToPrecision(timestamp, precision); + return new DateRangeBound(roundedLowerBound, precision); + } + + public static DateRangeBound upperBound(Instant timestamp, Precision precision) + { + return upperBound(ZonedDateTime.ofInstant(timestamp, ZoneOffset.UTC), precision); + } + + public static DateRangeBound upperBound(ZonedDateTime timestamp, Precision precision) + { + ZonedDateTime roundedUpperBound = DateRangeUtil.roundUpperBoundTimestampToPrecision(timestamp, precision); + return new DateRangeBound(roundedUpperBound, precision); + } + + public boolean isUnbounded() + { + return timestamp == null; + } + + public boolean isAfter(DateRangeBound other) + { + return isUnbounded() || other.isUnbounded() || timestamp.isAfter(other.timestamp); + } + + public Instant getTimestamp() + { + return timestamp.toInstant(); + } + + public Precision getPrecision() + { + return precision; + } + + @Override + public String toString() + { + if (isUnbounded()) + { + return "*"; + } + + return precision.formatter.format(timestamp); + } + + @Override + public boolean equals(Object obj) + { + if (obj == null || obj.getClass() != getClass()) + { + return false; + } + if (obj == this) + { + return true; + } + + DateRangeBound rhs = (DateRangeBound) obj; + return new EqualsBuilder() + .append(isUnbounded(), rhs.isUnbounded()) + .append(timestamp, rhs.timestamp) + .append(precision, rhs.precision) + .isEquals(); + } + + @Override + public int hashCode() + { + return Objects.hashCode(timestamp, precision); + } + + public enum Precision + { + YEAR(0x00, + new DateTimeFormatterBuilder() + .parseCaseSensitive() + .parseStrict() + .appendPattern("uuuu") + .parseDefaulting(MONTH_OF_YEAR, 1) + .parseDefaulting(DAY_OF_MONTH, 1) + .parseDefaulting(HOUR_OF_DAY, 0) + .parseDefaulting(MINUTE_OF_HOUR, 0) + .parseDefaulting(SECOND_OF_MINUTE, 0) + .parseDefaulting(MILLI_OF_SECOND, 0) + .toFormatter() + .withZone(ZoneOffset.UTC) + .withLocale(Locale.ROOT)), + + MONTH(0x01, + new DateTimeFormatterBuilder() + .parseCaseSensitive() + .parseStrict() + .appendPattern("uuuu-MM") + .parseDefaulting(DAY_OF_MONTH, 1) + .parseDefaulting(HOUR_OF_DAY, 0) + .parseDefaulting(MINUTE_OF_HOUR, 0) + .parseDefaulting(SECOND_OF_MINUTE, 0) + .parseDefaulting(MILLI_OF_SECOND, 0) + .toFormatter() + .withZone(ZoneOffset.UTC) + .withLocale(Locale.ROOT)), + + DAY(0x02, + new DateTimeFormatterBuilder() + .parseCaseSensitive() + .parseStrict() + .appendPattern("uuuu-MM-dd") + .parseDefaulting(HOUR_OF_DAY, 0) + .parseDefaulting(MINUTE_OF_HOUR, 0) + .parseDefaulting(SECOND_OF_MINUTE, 0) + .parseDefaulting(MILLI_OF_SECOND, 0) + .toFormatter() + .withZone(ZoneOffset.UTC) + .withLocale(Locale.ROOT)), + + HOUR(0x03, + new DateTimeFormatterBuilder() + .parseCaseSensitive() + .parseStrict() + .appendPattern("uuuu-MM-dd'T'HH") + .parseDefaulting(MINUTE_OF_HOUR, 0) + .parseDefaulting(SECOND_OF_MINUTE, 0) + .parseDefaulting(MILLI_OF_SECOND, 0) + .toFormatter() + .withZone(ZoneOffset.UTC) + .withLocale(Locale.ROOT)), + + MINUTE(0x04, + new DateTimeFormatterBuilder() + .parseCaseSensitive() + .parseStrict() + .appendPattern("uuuu-MM-dd'T'HH:mm") + .parseDefaulting(SECOND_OF_MINUTE, 0) + .parseDefaulting(MILLI_OF_SECOND, 0) + .toFormatter() + .withZone(ZoneOffset.UTC) + .withLocale(Locale.ROOT)), + + SECOND(0x05, + new DateTimeFormatterBuilder() + .parseCaseSensitive() + .parseStrict() + .appendPattern("uuuu-MM-dd'T'HH:mm:ss") + .parseDefaulting(MILLI_OF_SECOND, 0) + .toFormatter() + .withZone(ZoneOffset.UTC) + .withLocale(Locale.ROOT)), + + MILLISECOND(0x06, + new DateTimeFormatterBuilder() + .parseCaseSensitive() + .parseStrict() + .appendPattern("uuuu-MM-dd'T'HH:mm:ss.SSS") + .optionalStart() + .appendZoneId() + .optionalEnd() + .toFormatter() + .withZone(ZoneOffset.UTC) + .withLocale(Locale.ROOT)); + + private final int encoded; + private final DateTimeFormatter formatter; + + Precision(int encoded, DateTimeFormatter formatter) + { + this.encoded = encoded; + this.formatter = formatter; + } + + public int toEncoded() + { + return encoded; + } + + public static Precision fromEncoded(byte encoded) + { + for (Precision precision : values()) + { + if (precision.encoded == encoded) + { + return precision; + } + } + throw new IllegalArgumentException("Invalid precision encoding: " + encoded); + } + } + } + + public static class DateRangeBuilder + { + private DateRangeBound lowerBound = null; + private DateRangeBound upperBound = null; + + private DateRangeBuilder() {} + + public static DateRangeBuilder dateRange() + { + return new DateRangeBuilder(); + } + + public DateRangeBuilder withLowerBound(String lowerBound, DateRangeBound.Precision precision) + { + return withLowerBound(Instant.parse(lowerBound), precision); + } + + public DateRangeBuilder withUnboundedLowerBound() + { + this.lowerBound = DateRangeBound.UNBOUNDED; + return this; + } + + public DateRangeBuilder withUnboundedUpperBound() + { + this.upperBound = DateRangeBound.UNBOUNDED; + return this; + } + + public DateRangeBuilder withUpperBound(String upperBound, DateRangeBound.Precision precision) + { + return withUpperBound(Instant.parse(upperBound), precision); + } + + DateRangeBuilder withLowerBound(Instant lowerBound, DateRangeBound.Precision precision) + { + this.lowerBound = DateRangeBound.lowerBound(lowerBound, precision); + return this; + } + + DateRangeBuilder withUpperBound(Instant upperBound, DateRangeBound.Precision precision) + { + this.upperBound = DateRangeBound.upperBound(upperBound, precision); + return this; + } + + public DateRange build() + { + return new DateRange(this); + } + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/datetime/DateRangeUtil.java b/src/java/org/apache/cassandra/db/marshal/datetime/DateRangeUtil.java new file mode 100644 index 000000000000..8b73cfaa0899 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/datetime/DateRangeUtil.java @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal.datetime; + +import java.text.ParseException; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.temporal.ChronoField; +import java.util.Calendar; +import java.util.Locale; +import java.util.TimeZone; + +import org.apache.commons.lang3.StringUtils; + +import org.apache.cassandra.db.marshal.datetime.DateRange.DateRangeBound; + +import static java.time.temporal.TemporalAdjusters.firstDayOfMonth; +import static java.time.temporal.TemporalAdjusters.firstDayOfYear; +import static java.time.temporal.TemporalAdjusters.lastDayOfMonth; +import static java.time.temporal.TemporalAdjusters.lastDayOfYear; + +public class DateRangeUtil +{ + private static final int YEAR_LEVEL = 3; + private static final int[] FIELD_BY_LEVEL = + { + -1/*unused*/, -1, -1, Calendar.YEAR, Calendar.MONTH, Calendar.DAY_OF_MONTH, + Calendar.HOUR_OF_DAY, Calendar.MINUTE, Calendar.SECOND, Calendar.MILLISECOND + }; + private static final TimeZone UTC_TIME_ZONE = TimeZone.getTimeZone("UTC"); + + public static DateRange parseDateRange(String source) throws ParseException + { + if (StringUtils.isBlank(source)) + { + throw new IllegalArgumentException("Date range is null or blank"); + } + if (source.charAt(0) == '[') + { + if (source.charAt(source.length() - 1) != ']') + { + throw new IllegalArgumentException("If date range starts with [ must end with ]; got " + source); + } + int middle = source.indexOf(" TO "); + if (middle < 0) + { + throw new IllegalArgumentException("If date range starts with [ must contain ' TO '; got " + source); + } + String lowerBoundString = source.substring(1, middle); + String upperBoundString = source.substring(middle + " TO ".length(), source.length() - 1); + return new DateRange(parseLowerBound(lowerBoundString), parseUpperBound(upperBoundString)); + } + else + { + return new DateRange(parseLowerBound(source)); + } + } + + public static ZonedDateTime roundUpperBoundTimestampToPrecision(ZonedDateTime timestamp, DateRangeBound.Precision precision) + { + switch (precision) + { + case YEAR: + timestamp = timestamp.with(lastDayOfYear()); + case MONTH: + timestamp = timestamp.with(lastDayOfMonth()); + case DAY: + timestamp = timestamp.with(ChronoField.HOUR_OF_DAY, 23); + case HOUR: + timestamp = timestamp.with(ChronoField.MINUTE_OF_HOUR, 59); + case MINUTE: + timestamp = timestamp.with(ChronoField.SECOND_OF_MINUTE, 59); + case SECOND: + timestamp = timestamp.with(ChronoField.MILLI_OF_SECOND, 999); + case MILLISECOND: + // DateRangeField ignores any precision beyond milliseconds + return timestamp; + default: + throw new IllegalStateException("Unsupported date time precision for the upper bound: " + precision); + } + } + + public static ZonedDateTime roundLowerBoundTimestampToPrecision(ZonedDateTime timestamp, DateRangeBound.Precision precision) + { + switch (precision) + { + case YEAR: + timestamp = timestamp.with(firstDayOfYear()); + case MONTH: + timestamp = timestamp.with(firstDayOfMonth()); + case DAY: + timestamp = timestamp.with(ChronoField.HOUR_OF_DAY, 0); + case HOUR: + timestamp = timestamp.with(ChronoField.MINUTE_OF_HOUR, 0); + case MINUTE: + timestamp = timestamp.with(ChronoField.SECOND_OF_MINUTE, 0); + case SECOND: + timestamp = timestamp.with(ChronoField.MILLI_OF_SECOND, 0); + case MILLISECOND: + // DateRangeField ignores any precision beyond milliseconds + return timestamp; + default: + throw new IllegalStateException("Unsupported date time precision for the upper bound: " + precision); + } + } + + private static DateRangeBound parseLowerBound(String source) throws ParseException + { + Calendar lowerBoundCalendar = parseCalendar(source); + int calPrecisionField = getCalPrecisionField(lowerBoundCalendar); + if (calPrecisionField < 0) + { + return DateRangeBound.UNBOUNDED; + } + return DateRangeBound.lowerBound(toZonedDateTime(lowerBoundCalendar), getCalendarPrecision(calPrecisionField)); + } + + private static DateRangeBound parseUpperBound(String source) throws ParseException + { + Calendar upperBoundCalendar = parseCalendar(source); + int calPrecisionField = getCalPrecisionField(upperBoundCalendar); + if (calPrecisionField < 0) + { + return DateRangeBound.UNBOUNDED; + } + ZonedDateTime upperBoundDateTime = toZonedDateTime(upperBoundCalendar); + DateRangeBound.Precision precision = getCalendarPrecision(calPrecisionField); + return DateRangeBound.upperBound(upperBoundDateTime, precision); + } + + /** + * This method was extracted from org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree + * (Apache Lucene™) for compatibility with DSE. + * The class is distributed under Apache-2.0 License attached to this release. + * + * Calendar utility method: + * Gets the Calendar field code of the last field that is set prior to an unset field. It only + * examines fields relevant to the prefix tree. If no fields are set, it returns -1. */ + private static int getCalPrecisionField(Calendar cal) { + int lastField = -1; + for (int level = YEAR_LEVEL; level < FIELD_BY_LEVEL.length; level++) { + int field = FIELD_BY_LEVEL[level]; + if (!cal.isSet(field)) + break; + lastField = field; + } + return lastField; + } + + /** + * This method was extracted from org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree + * (Apache Lucene™) for compatibility with DSE. + * The class is distributed under Apache-2.0 License attached to this release. + * + * Calendar utility method: + * It will only set the fields found, leaving + * the remainder in an un-set state. A leading '-' or '+' is optional (positive assumed), and a + * trailing 'Z' is also optional. + * @param str not null and not empty + * @return not null + */ + private static Calendar parseCalendar(String str) throws ParseException { + // example: +2014-10-23T21:22:33.159Z + if (str == null || str.isEmpty()) + throw new IllegalArgumentException("str is null or blank"); + Calendar cal = Calendar.getInstance(UTC_TIME_ZONE, Locale.ROOT); + cal.clear(); + if (str.equals("*")) + return cal; + int offset = 0;//a pointer + try { + //year & era: + int lastOffset = str.charAt(str.length()-1) == 'Z' ? str.length() - 1 : str.length(); + int hyphenIdx = str.indexOf('-', 1);//look past possible leading hyphen + if (hyphenIdx < 0) + hyphenIdx = lastOffset; + int year = Integer.parseInt(str.substring(offset, hyphenIdx)); + cal.set(Calendar.ERA, year <= 0 ? 0 : 1); + cal.set(Calendar.YEAR, year <= 0 ? -1*year + 1 : year); + offset = hyphenIdx + 1; + if (lastOffset < offset) + return cal; + + //NOTE: We aren't validating separator chars, and we unintentionally accept leading +/-. + // The str.substring()'s hopefully get optimized to be stack-allocated. + + //month: + cal.set(Calendar.MONTH, Integer.parseInt(str.substring(offset, offset+2)) - 1);//starts at 0 + offset += 3; + if (lastOffset < offset) + return cal; + //day: + cal.set(Calendar.DAY_OF_MONTH, Integer.parseInt(str.substring(offset, offset+2))); + offset += 3; + if (lastOffset < offset) + return cal; + //hour: + cal.set(Calendar.HOUR_OF_DAY, Integer.parseInt(str.substring(offset, offset+2))); + offset += 3; + if (lastOffset < offset) + return cal; + //minute: + cal.set(Calendar.MINUTE, Integer.parseInt(str.substring(offset, offset+2))); + offset += 3; + if (lastOffset < offset) + return cal; + //second: + cal.set(Calendar.SECOND, Integer.parseInt(str.substring(offset, offset+2))); + offset += 3; + if (lastOffset < offset) + return cal; + //ms: + cal.set(Calendar.MILLISECOND, Integer.parseInt(str.substring(offset, offset+3))); + offset += 3;//last one, move to next char + if (lastOffset == offset) + return cal; + } catch (Exception e) { + ParseException pe = new ParseException("Improperly formatted date: "+str, offset); + pe.initCause(e); + throw pe; + } + throw new ParseException("Improperly formatted date: "+str, offset); + } + + private static DateRangeBound.Precision getCalendarPrecision(int calendarPrecision) + { + switch (calendarPrecision) + { + case Calendar.YEAR: + return DateRangeBound.Precision.YEAR; + case Calendar.MONTH: + return DateRangeBound.Precision.MONTH; + case Calendar.DAY_OF_MONTH: + return DateRangeBound.Precision.DAY; + case Calendar.HOUR_OF_DAY: + return DateRangeBound.Precision.HOUR; + case Calendar.MINUTE: + return DateRangeBound.Precision.MINUTE; + case Calendar.SECOND: + return DateRangeBound.Precision.SECOND; + case Calendar.MILLISECOND: + return DateRangeBound.Precision.MILLISECOND; + default: + throw new IllegalStateException("Unsupported date time precision: " + calendarPrecision); + } + } + + private static ZonedDateTime toZonedDateTime(Calendar calendar) + { + int year = calendar.get(Calendar.YEAR); + if (calendar.get(Calendar.ERA) == 0) + { + // BC era; 1 BC == 0 AD, 0 BD == -1 AD, etc + year -= 1; + if (year > 0) + { + year = -year; + } + } + LocalDateTime localDateTime = LocalDateTime.of(year, + calendar.get(Calendar.MONTH) + 1, + calendar.get(Calendar.DAY_OF_MONTH), + calendar.get(Calendar.HOUR_OF_DAY), + calendar.get(Calendar.MINUTE), + calendar.get(Calendar.SECOND)); + localDateTime = localDateTime.with(ChronoField.MILLI_OF_SECOND, calendar.get(Calendar.MILLISECOND)); + return ZonedDateTime.of(localDateTime, ZoneOffset.UTC); + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/geometry/GeometricType.java b/src/java/org/apache/cassandra/db/marshal/geometry/GeometricType.java new file mode 100644 index 000000000000..2516d7d23bda --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/geometry/GeometricType.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal.geometry; + +public enum GeometricType +{ + POINT(Point.class, Point.serializer), + LINESTRING(LineString.class, LineString.serializer), + POLYGON(Polygon.class, Polygon.serializer); + + private final Class geoClass; + private final OgcGeometry.Serializer serializer; + + GeometricType(Class geoClass, OgcGeometry.Serializer serializer) + { + this.geoClass = geoClass; + this.serializer = serializer; + } + + public Class getGeoClass() + { + return geoClass; + } + + public OgcGeometry.Serializer getSerializer() + { + return serializer; + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/geometry/LineString.java b/src/java/org/apache/cassandra/db/marshal/geometry/LineString.java new file mode 100644 index 000000000000..a31854fa9100 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/geometry/LineString.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal.geometry; + +import java.nio.ByteBuffer; + +import com.esri.core.geometry.GeoJsonExportFlags; +import com.esri.core.geometry.Operator; +import com.esri.core.geometry.OperatorExportToGeoJson; +import com.esri.core.geometry.OperatorFactoryLocal; +import com.esri.core.geometry.ogc.OGCGeometry; +import com.esri.core.geometry.ogc.OGCLineString; +import org.apache.cassandra.serializers.MarshalException; + +public class LineString extends OgcGeometry +{ + public static final Serializer serializer = new Serializer() + { + @Override + public String toWellKnownText(LineString geometry) + { + return geometry.lineString.asText(); + } + + @Override + public ByteBuffer toWellKnownBinaryNativeOrder(LineString geometry) + { + return geometry.lineString.asBinary(); + } + + @Override + public String toGeoJson(LineString geometry) + { + OperatorExportToGeoJson op = (OperatorExportToGeoJson) OperatorFactoryLocal.getInstance().getOperator(Operator.Type.ExportToGeoJson); + return op.execute(GeoJsonExportFlags.geoJsonExportSkipCRS, geometry.lineString.esriSR, geometry.lineString.getEsriGeometry()); + } + + @Override + public LineString fromWellKnownText(String source) + { + return new LineString(fromOgcWellKnownText(source, OGCLineString.class)); + } + + @Override + public LineString fromWellKnownBinary(ByteBuffer source) + { + return new LineString(fromOgcWellKnownBinary(source, OGCLineString.class)); + } + + @Override + public LineString fromGeoJson(String source) + { + return new LineString(fromOgcGeoJson(source, OGCLineString.class)); + } + }; + + private final OGCLineString lineString; + + public LineString(OGCLineString lineString) + { + this.lineString = lineString; + validate(); + } + + @Override + public GeometricType getType() + { + return GeometricType.LINESTRING; + } + + @Override + public void validate() throws MarshalException + { + validateOgcGeometry(lineString); + } + + @Override + public Serializer getSerializer() + { + return serializer; + } + + @Override + protected OGCGeometry getOgcGeometry() + { + return lineString; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + LineString that = (LineString) o; + + return !(lineString != null ? !lineString.equals(that.lineString) : that.lineString != null); + + } + + @Override + public int hashCode() + { + return lineString != null ? lineString.hashCode() : 0; + } + + @Override + public String toString() + { + return asWellKnownText(); + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/geometry/OgcGeometry.java b/src/java/org/apache/cassandra/db/marshal/geometry/OgcGeometry.java new file mode 100644 index 000000000000..6f3be38e7415 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/geometry/OgcGeometry.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal.geometry; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import com.esri.core.geometry.GeometryException; +import com.esri.core.geometry.JsonGeometryException; +import com.esri.core.geometry.SpatialReference; +import com.esri.core.geometry.ogc.OGCGeometry; +import org.apache.cassandra.serializers.MarshalException; + +public abstract class OgcGeometry +{ + + // default spatial reference for wkt/wkb + public static final SpatialReference SPATIAL_REFERENCE_4326 = SpatialReference.create(4326); + + public interface Serializer + { + String toWellKnownText(T geometry); + + // We need to return a Big Endian ByteBuffer as that's required by org.apache.cassandra.db.NativeDecoratedKey + // when the memtable allocation type is "offheap_objects". See https://datastax.jira.com/browse/DSP-16302 + // Note that the order set here may not match the actual endianess. OGC serialization encodes actual endianess + // and discards BB order set here. + default ByteBuffer toWellKnownBinary(T geometry) + { + return toWellKnownBinaryNativeOrder(geometry).order(ByteOrder.BIG_ENDIAN); + } + + ByteBuffer toWellKnownBinaryNativeOrder(T geometry); + + String toGeoJson(T geometry); + + T fromWellKnownText(String source); + + T fromWellKnownBinary(ByteBuffer source); + + T fromGeoJson(String source); + } + + public abstract GeometricType getType(); + + public abstract void validate() throws MarshalException; + + public abstract Serializer getSerializer(); + + static void validateType(OGCGeometry geometry, Class klass) + { + if (!geometry.getClass().equals(klass)) + { + throw new MarshalException(String.format("%s is not of type %s", + geometry.getClass().getSimpleName(), + klass.getSimpleName())); + } + } + + static ByteBuffer getWkb(OGCGeometry geometry) + { + try + { + return geometry.asBinary(); + } + catch (GeometryException | IllegalArgumentException e) + { + throw new MarshalException("Invalid Geometry", e); + } + } + + static String getWkt(OGCGeometry geometry) + { + try + { + return geometry.asText(); + } + catch (GeometryException | IllegalArgumentException e) + { + throw new MarshalException("Invalid Geometry", e); + } + } + + static void validateNormalization(OGCGeometry geometry, ByteBuffer source) + { + ByteBuffer normalized = getWkb(geometry); + ByteBuffer inputCopy = source.slice(); + + // since the data we get is sometimes part of a longer string of bytes, we set the limit to the normalized + // buffer length. Normalization only ever adds and rearranges points though, so this should be ok + if (inputCopy.remaining() > normalized.remaining()) + { + inputCopy.limit(normalized.remaining()); + } + + if (!normalized.equals(inputCopy)) + { + String klass = geometry.getClass().getSimpleName(); + String msg = String.format("%s is not normalized. %s should be defined/serialized as: %s", klass, klass, getWkt(geometry)); + throw new MarshalException(msg); + } + } + + static T fromOgcWellKnownText(String source, Class klass) + { + OGCGeometry geometry; + try + { + geometry = OGCGeometry.fromText(source); + } + catch (IllegalArgumentException e) + { + throw new MarshalException(e.getMessage()); + } + validateType(geometry, klass); + return (T) geometry; + } + + static T fromOgcWellKnownBinary(ByteBuffer source, Class klass) + { + OGCGeometry geometry; + try + { + geometry = OGCGeometry.fromBinary(source); + } + catch (IllegalArgumentException e) + { + throw new MarshalException(e.getMessage()); + } + validateType(geometry, klass); + validateNormalization(geometry, source); + return (T) geometry; + } + + static T fromOgcGeoJson(String source, Class klass) + { + OGCGeometry geometry; + try + { + geometry = OGCGeometry.fromGeoJson(source); + } + catch (IllegalArgumentException | JsonGeometryException e) + { + throw new MarshalException(e.getMessage()); + } + validateType(geometry, klass); + return (T) geometry; + } + + public boolean contains(OgcGeometry geometry) + { + if (!(geometry instanceof OgcGeometry)) + { + throw new UnsupportedOperationException(String.format("%s is not compatible with %s.contains", + geometry.getClass().getSimpleName(), getClass().getSimpleName())); + } + + OGCGeometry thisGeometry = getOgcGeometry(); + OGCGeometry thatGeometry = ((OgcGeometry) geometry).getOgcGeometry(); + if (thisGeometry != null && thatGeometry != null) + { + return thisGeometry.contains(thatGeometry); + } + else + { + return false; + } + } + + protected abstract OGCGeometry getOgcGeometry(); + + static void validateOgcGeometry(OGCGeometry geometry) + { + try + { + if (geometry.is3D()) + { + throw new MarshalException(String.format("'%s' is not 2D", getWkt(geometry))); + } + + if (!geometry.isSimple()) + { + throw new MarshalException(String.format("'%s' is not simple. Points and edges cannot self-intersect.", getWkt(geometry))); + } + } + catch (GeometryException e) + { + throw new MarshalException("Invalid geometry", e); + } + } + + public String asWellKnownText() + { + return getSerializer().toWellKnownText(this); + } + + public ByteBuffer asWellKnownBinary() + { + return getSerializer().toWellKnownBinary(this); + } + + public String asGeoJson() + { + return getSerializer().toGeoJson(this); + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/geometry/Point.java b/src/java/org/apache/cassandra/db/marshal/geometry/Point.java new file mode 100644 index 000000000000..0992bd768725 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/geometry/Point.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal.geometry; + +import java.nio.ByteBuffer; + +import com.esri.core.geometry.GeoJsonExportFlags; +import com.esri.core.geometry.Operator; +import com.esri.core.geometry.OperatorExportToGeoJson; +import com.esri.core.geometry.OperatorFactoryLocal; +import com.esri.core.geometry.ogc.OGCGeometry; +import com.esri.core.geometry.ogc.OGCPoint; +import org.apache.cassandra.serializers.MarshalException; + +public class Point extends OgcGeometry +{ + public static final Serializer serializer = new Serializer() + { + @Override + public String toWellKnownText(Point geometry) + { + return geometry.point.asText(); + } + + @Override + public ByteBuffer toWellKnownBinaryNativeOrder(Point geometry) + { + return geometry.point.asBinary(); + } + + @Override + public String toGeoJson(Point geometry) + { + OperatorExportToGeoJson op = (OperatorExportToGeoJson) OperatorFactoryLocal.getInstance().getOperator(Operator.Type.ExportToGeoJson); + return op.execute(GeoJsonExportFlags.geoJsonExportSkipCRS, geometry.point.esriSR, geometry.point.getEsriGeometry()); + } + + @Override + public Point fromWellKnownText(String source) + { + return new Point(fromOgcWellKnownText(source, OGCPoint.class)); + } + + @Override + public Point fromWellKnownBinary(ByteBuffer source) + { + return new Point(fromOgcWellKnownBinary(source, OGCPoint.class)); + } + + @Override + public Point fromGeoJson(String source) + { + return new Point(fromOgcGeoJson(source, OGCPoint.class)); + } + }; + + final OGCPoint point; + + public Point(double x, double y) + { + this(new OGCPoint(new com.esri.core.geometry.Point(x, y), OgcGeometry.SPATIAL_REFERENCE_4326)); + } + + private Point(OGCPoint point) + { + this.point = point; + validate(); + } + + @Override + public boolean contains(OgcGeometry geometry) + { + return false; + } + + @Override + public GeometricType getType() + { + return GeometricType.POINT; + } + + @Override + public void validate() throws MarshalException + { + validateOgcGeometry(point); + if (point.isEmpty() || point.is3D()) + throw new MarshalException(getClass().getSimpleName() + " requires exactly 2 coordinate values"); + } + + @Override + protected OGCGeometry getOgcGeometry() + { + return point; + } + + @Override + public Serializer getSerializer() + { + return serializer; + } + + public OGCPoint getOgcPoint() + { + return point; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Point point1 = (Point) o; + + return !(point != null ? !point.equals(point1.point) : point1.point != null); + + } + + @Override + public int hashCode() + { + return point != null ? point.hashCode() : 0; + } + + @Override + public String toString() + { + return asWellKnownText(); + } +} diff --git a/src/java/org/apache/cassandra/db/marshal/geometry/Polygon.java b/src/java/org/apache/cassandra/db/marshal/geometry/Polygon.java new file mode 100644 index 000000000000..d51181566d04 --- /dev/null +++ b/src/java/org/apache/cassandra/db/marshal/geometry/Polygon.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.marshal.geometry; + +import java.nio.ByteBuffer; + +import com.esri.core.geometry.GeoJsonExportFlags; +import com.esri.core.geometry.Operator; +import com.esri.core.geometry.OperatorExportToGeoJson; +import com.esri.core.geometry.OperatorFactoryLocal; +import com.esri.core.geometry.ogc.OGCGeometry; +import com.esri.core.geometry.ogc.OGCPolygon; +import org.apache.cassandra.serializers.MarshalException; + +public class Polygon extends OgcGeometry +{ + public static final Serializer serializer = new Serializer() + { + @Override + public String toWellKnownText(Polygon geometry) + { + return geometry.polygon.asText(); + } + + @Override + public ByteBuffer toWellKnownBinaryNativeOrder(Polygon geometry) + { + return geometry.polygon.asBinary(); + } + + @Override + public String toGeoJson(Polygon geometry) + { + OperatorExportToGeoJson op = (OperatorExportToGeoJson) OperatorFactoryLocal.getInstance().getOperator(Operator.Type.ExportToGeoJson); + return op.execute(GeoJsonExportFlags.geoJsonExportSkipCRS, geometry.polygon.esriSR, geometry.polygon.getEsriGeometry()); + } + + @Override + public Polygon fromWellKnownText(String source) + { + return new Polygon(fromOgcWellKnownText(source, OGCPolygon.class)); + } + + @Override + public Polygon fromWellKnownBinary(ByteBuffer source) + { + return new Polygon(fromOgcWellKnownBinary(source, OGCPolygon.class)); + } + + @Override + public Polygon fromGeoJson(String source) + { + return new Polygon(fromOgcGeoJson(source, OGCPolygon.class)); + } + }; + + OGCPolygon polygon; + + public Polygon(OGCPolygon polygon) + { + this.polygon = polygon; + validate(); + } + + @Override + protected OGCGeometry getOgcGeometry() + { + return polygon; + } + + @Override + public GeometricType getType() + { + return GeometricType.POLYGON; + } + + @Override + public void validate() throws MarshalException + { + validateOgcGeometry(polygon); + } + + @Override + public Serializer getSerializer() + { + return serializer; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Polygon polygon1 = (Polygon) o; + + return !(polygon != null ? !polygon.equals(polygon1.polygon) : polygon1.polygon != null); + + } + + @Override + public int hashCode() + { + return polygon != null ? polygon.hashCode() : 0; + } + + @Override + public String toString() + { + return asWellKnownText(); + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/AbstractAllocatorMemtable.java b/src/java/org/apache/cassandra/db/memtable/AbstractAllocatorMemtable.java new file mode 100644 index 000000000000..6636a4783f0d --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/AbstractAllocatorMemtable.java @@ -0,0 +1,295 @@ +/* + * 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.memtable; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.WrappedRunnable; +import org.apache.cassandra.utils.concurrent.OpOrder; +import org.apache.cassandra.utils.memory.HeapPool; +import org.apache.cassandra.utils.memory.MemtableAllocator; +import org.apache.cassandra.utils.memory.MemtableCleaner; +import org.apache.cassandra.utils.memory.MemtablePool; +import org.apache.cassandra.utils.memory.NativePool; +import org.apache.cassandra.utils.memory.SlabPool; + +/** + * A memtable that uses memory tracked and maybe allocated via a MemtableAllocator from a MemtablePool. + * Provides methods of memory tracking and triggering flushes when the relevant limits are reached. + */ +public abstract class AbstractAllocatorMemtable extends AbstractMemtableWithCommitlog +{ + private static final Logger logger = LoggerFactory.getLogger(AbstractAllocatorMemtable.class); + + public static final MemtablePool MEMORY_POOL = AbstractAllocatorMemtable.createMemtableAllocatorPool(); + + protected final Owner owner; + protected final MemtableAllocator allocator; + + // Record the comparator of the CFS at the creation of the memtable. This + // is only used when a user update the CF comparator, to know if the + // memtable was created with the new or old comparator. + public final ClusteringComparator initialComparator; + + private final long creationNano = System.nanoTime(); + + private static MemtablePool createMemtableAllocatorPool() + { + long heapLimit = DatabaseDescriptor.getMemtableHeapSpaceInMb() << 20; + long offHeapLimit = DatabaseDescriptor.getMemtableOffheapSpaceInMb() << 20; + float memtableCleanupThreshold = DatabaseDescriptor.getMemtableCleanupThreshold(); + MemtableCleaner cleaner = AbstractAllocatorMemtable::flushLargestMemtable; + switch (DatabaseDescriptor.getMemtableAllocationType()) + { + case unslabbed_heap_buffers: + logger.debug("Memtables allocating with on-heap buffers"); + return new HeapPool(heapLimit, memtableCleanupThreshold, cleaner); + case heap_buffers: + logger.debug("Memtables allocating with on-heap slabs"); + return new SlabPool(heapLimit, 0, memtableCleanupThreshold, cleaner); + case offheap_buffers: + logger.debug("Memtables allocating with off-heap buffers"); + return new SlabPool(heapLimit, offHeapLimit, memtableCleanupThreshold, cleaner); + case offheap_objects: + logger.debug("Memtables allocating with off-heap objects"); + return new NativePool(heapLimit, offHeapLimit, memtableCleanupThreshold, cleaner); + default: + throw new AssertionError(); + } + } + + // only to be used by init(), to setup the very first memtable for the cfs + public AbstractAllocatorMemtable(AtomicReference commitLogLowerBound, TableMetadataRef metadataRef, Owner owner) + { + super(metadataRef, commitLogLowerBound); + this.allocator = MEMORY_POOL.newAllocator(); + this.initialComparator = metadata.get().comparator; + this.owner = owner; + scheduleFlush(); + } + + public MemtableAllocator getAllocator() + { + return allocator; + } + + public boolean shouldSwitch(ColumnFamilyStore.FlushReason reason) + { + switch (reason) + { + case SCHEMA_CHANGE: + return initialComparator != metadata().comparator // If the CF comparator has changed, because our partitions reference the old one + || metadata().params.memtable.factory != factory(); // If a different type of memtable is requested + default: + return true; + } + } + + public void metadataUpdated() + { + scheduleFlush(); + } + + public void performSnapshot(String snapshotName) + { + // unless shouldSwitch(SNAPSHOT) returns false, this cannot be called. + throw new AssertionError(); + } + + protected abstract Factory factory(); + + public void switchOut(OpOrder.Barrier writeBarrier, AtomicReference commitLogUpperBound) + { + super.switchOut(writeBarrier, commitLogUpperBound); + allocator.setDiscarding(); + } + + public void discard() + { + super.discard(); + allocator.setDiscarded(); + } + + public String toString() + { + MemoryUsage usage = Memtable.getMemoryUsage(this); + return String.format("Memtable-%s@%s(%s serialized bytes, %s ops, %s)", + metadata.get().name, + hashCode(), + FBUtilities.prettyPrintMemory(getLiveDataSize()), + getOperations(), + usage); + } + + @Override + public void addMemoryUsageTo(MemoryUsage stats) + { + stats.ownershipRatioOnHeap += getAllocator().onHeap().ownershipRatio(); + stats.ownershipRatioOffHeap += getAllocator().offHeap().ownershipRatio(); + stats.ownsOnHeap += getAllocator().onHeap().owns(); + stats.ownsOffHeap += getAllocator().offHeap().owns(); + } + + public void markExtraOnHeapUsed(long additionalSpace, OpOrder.Group opGroup) + { + getAllocator().onHeap().allocate(additionalSpace, opGroup); + } + + public void markExtraOffHeapUsed(long additionalSpace, OpOrder.Group opGroup) + { + getAllocator().offHeap().allocate(additionalSpace, opGroup); + } + + void scheduleFlush() + { + int period = metadata().params.memtableFlushPeriodInMs; + if (period > 0) + scheduleFlush(owner, period); + } + + private static void scheduleFlush(Owner owner, int period) + { + logger.trace("scheduling flush in {} ms", period); + WrappedRunnable runnable = new WrappedRunnable() + { + protected void runMayThrow() + { + Memtable current = owner.getCurrentMemtable(); + if (current instanceof AbstractAllocatorMemtable) + ((AbstractAllocatorMemtable) current).flushIfPeriodExpired(); + } + }; + ScheduledExecutors.scheduledTasks.schedule(runnable, period, TimeUnit.MILLISECONDS); + } + + private void flushIfPeriodExpired() + { + int period = metadata().params.memtableFlushPeriodInMs; + if (period > 0 && (System.nanoTime() - creationNano >= TimeUnit.MILLISECONDS.toNanos(period))) + { + if (isClean()) + { + // if we're still clean, instead of swapping just reschedule a flush for later + scheduleFlush(owner, period); + } + else + { + // we'll be rescheduled by the constructor of the Memtable. + owner.signalFlushRequired(AbstractAllocatorMemtable.this, + ColumnFamilyStore.FlushReason.MEMTABLE_PERIOD_EXPIRED); + } + } + } + + /** + * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately + * queues it for flushing. If the memtable selected is flushed before this completes, no work is done. + */ + public static CompletableFuture flushLargestMemtable() + { + float largestRatio = 0f; + AbstractAllocatorMemtable largestMemtable = null; + Memtable.MemoryUsage largestUsage = null; + float liveOnHeap = 0, liveOffHeap = 0; + // we take a reference to the current main memtable for the CF prior to snapping its ownership ratios + // to ensure we have some ordering guarantee for performing the switchMemtableIf(), i.e. we will only + // swap if the memtables we are measuring here haven't already been swapped by the time we try to swap them + for (Memtable currentMemtable : ColumnFamilyStore.activeMemtables()) + { + if (!(currentMemtable instanceof AbstractAllocatorMemtable)) + continue; + AbstractAllocatorMemtable current = (AbstractAllocatorMemtable) currentMemtable; + + // find the total ownership ratio for the memtable and all SecondaryIndexes owned by this CF, + // both on- and off-heap, and select the largest of the two ratios to weight this CF + MemoryUsage usage = Memtable.newMemoryUsage(); + current.addMemoryUsageTo(usage); + + for (Memtable indexMemtable : current.owner.getIndexMemtables()) + if (indexMemtable instanceof AbstractAllocatorMemtable) + indexMemtable.addMemoryUsageTo(usage); + + float ratio = Math.max(usage.ownershipRatioOnHeap, usage.ownershipRatioOffHeap); + if (ratio > largestRatio) + { + largestMemtable = current; + largestUsage = usage; + largestRatio = ratio; + } + + liveOnHeap += usage.ownershipRatioOnHeap; + liveOffHeap += usage.ownershipRatioOffHeap; + } + + CompletableFuture returnFuture = new CompletableFuture<>(); + + if (largestMemtable != null) + { + float usedOnHeap = MEMORY_POOL.onHeap.usedRatio(); + float usedOffHeap = MEMORY_POOL.offHeap.usedRatio(); + float flushingOnHeap = MEMORY_POOL.onHeap.reclaimingRatio(); + float flushingOffHeap = MEMORY_POOL.offHeap.reclaimingRatio(); + logger.debug("Flushing largest {} to free up room. Used total: {}, live: {}, flushing: {}, this: {}", + largestMemtable.owner, ratio(usedOnHeap, usedOffHeap), ratio(liveOnHeap, liveOffHeap), + ratio(flushingOnHeap, flushingOffHeap), ratio(largestUsage.ownershipRatioOnHeap, largestUsage.ownershipRatioOffHeap)); + + ListenableFuture flushFuture = largestMemtable.owner.signalFlushRequired(largestMemtable, ColumnFamilyStore.FlushReason.MEMTABLE_LIMIT); + flushFuture.addListener(() -> { + try + { + flushFuture.get(); + returnFuture.complete(true); + } + catch (Throwable t) + { + returnFuture.completeExceptionally(t); + } + }, MoreExecutors.directExecutor()); + } + else + { + logger.debug("Flushing of largest memtable, not done, no memtable found"); + + returnFuture.complete(false); + } + + return returnFuture; + } + + private static String ratio(float onHeap, float offHeap) + { + return String.format("%.2f/%.2f", onHeap, offHeap); + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/AbstractMemtable.java b/src/java/org/apache/cassandra/db/memtable/AbstractMemtable.java new file mode 100644 index 000000000000..b6d4a13b2031 --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/AbstractMemtable.java @@ -0,0 +1,210 @@ +/* + * 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.memtable; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableMetadataRef; + +public abstract class AbstractMemtable implements Memtable +{ + protected final AtomicLong currentOperations = new AtomicLong(0); + protected final ColumnsCollector columnsCollector; + protected final StatsCollector statsCollector = new StatsCollector(); + // The smallest timestamp for all partitions stored in this memtable + protected AtomicLong minTimestamp = new AtomicLong(Long.MAX_VALUE); + private final AtomicReference flushTransaction = new AtomicReference<>(null); + protected TableMetadataRef metadata; + + public AbstractMemtable(TableMetadataRef metadataRef) + { + this.metadata = metadataRef; + this.columnsCollector = new ColumnsCollector(metadata.get().regularAndStaticColumns()); + } + + public TableMetadata metadata() + { + return metadata.get(); + } + + public long getOperations() + { + return currentOperations.get(); + } + + public long getMinTimestamp() + { + return minTimestamp.get(); + } + + protected void updateMin(AtomicLong minTracker, long newValue) + { + while (true) + { + long memtableMinTimestamp = minTracker.get(); + if (memtableMinTimestamp <= newValue) + break; + if (minTracker.compareAndSet(memtableMinTimestamp, newValue)) + break; + } + } + + RegularAndStaticColumns columns() + { + return columnsCollector.get(); + } + + EncodingStats encodingStats() + { + return statsCollector.get(); + } + + public LifecycleTransaction getFlushTransaction() + { + return flushTransaction.get(); + } + + public LifecycleTransaction setFlushTransaction(LifecycleTransaction flushTransaction) + { + return this.flushTransaction.getAndSet(flushTransaction); + } + + protected static class ColumnsCollector + { + private final HashMap predefined = new HashMap<>(); + private final ConcurrentSkipListSet extra = new ConcurrentSkipListSet<>(); + + ColumnsCollector(RegularAndStaticColumns columns) + { + for (ColumnMetadata def : columns.statics) + predefined.put(def, new AtomicBoolean()); + for (ColumnMetadata def : columns.regulars) + predefined.put(def, new AtomicBoolean()); + } + + public void update(RegularAndStaticColumns columns) + { + for (ColumnMetadata s : columns.statics) + update(s); + for (ColumnMetadata r : columns.regulars) + update(r); + } + + public void update(ColumnsCollector other) + { + for (Map.Entry v : other.predefined.entrySet()) + if (v.getValue().get()) + update(v.getKey()); + + extra.addAll(other.extra); + } + + private void update(ColumnMetadata definition) + { + AtomicBoolean present = predefined.get(definition); + if (present != null) + { + if (!present.get()) + present.set(true); + } + else + { + extra.add(definition); + } + } + + /** + * Get the current state of the columns set. + * + * Note: If this is executed while mutations are still being performed on the table (e.g. to prepare + * an sstable for streaming when Memtable.Factory.streamFromMemtable() is true), the resulting view may be + * in a somewhat inconsistent state (it may include partial updates, as well as miss updates older than + * ones it does include). + */ + public RegularAndStaticColumns get() + { + RegularAndStaticColumns.Builder builder = RegularAndStaticColumns.builder(); + for (Map.Entry e : predefined.entrySet()) + if (e.getValue().get()) + builder.add(e.getKey()); + return builder.addAll(extra).build(); + } + } + + protected static class StatsCollector + { + private final AtomicReference stats = new AtomicReference<>(EncodingStats.NO_STATS); + + public void update(EncodingStats newStats) + { + while (true) + { + EncodingStats current = stats.get(); + EncodingStats updated = current.mergeWith(newStats); + if (stats.compareAndSet(current, updated)) + return; + } + } + + public EncodingStats get() + { + return stats.get(); + } + } + + protected abstract class AbstractFlushCollection

    implements FlushCollection

    + { + public long dataSize() + { + return getLiveDataSize(); + } + + public CommitLogPosition commitLogLowerBound() + { + return AbstractMemtable.this.getCommitLogLowerBound(); + } + + public LastCommitLogPosition commitLogUpperBound() + { + return AbstractMemtable.this.getFinalCommitLogUpperBound(); + } + + public EncodingStats encodingStats() + { + return AbstractMemtable.this.encodingStats(); + } + + public RegularAndStaticColumns columns() + { + return AbstractMemtable.this.columns(); + } + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/AbstractMemtableWithCommitlog.java b/src/java/org/apache/cassandra/db/memtable/AbstractMemtableWithCommitlog.java new file mode 100644 index 000000000000..a782fece2be2 --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/AbstractMemtableWithCommitlog.java @@ -0,0 +1,126 @@ +/* + * 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.memtable; + +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.cassandra.db.commitlog.CommitLog; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.concurrent.OpOrder; + +/** + * Memtable that uses a commit log for persistence. Provides methods of tracking the commit log positions covered by + * it and safely switching between memtables. + */ +public abstract class AbstractMemtableWithCommitlog extends AbstractMemtable +{ + // The approximate lower bound by this memtable; must be <= commitLogLowerBound once our predecessor + // has been finalised, and this is enforced in the ColumnFamilyStore.setCommitLogUpperBound + private final CommitLogPosition approximateCommitLogLowerBound = CommitLog.instance.getCurrentPosition(); + // the precise lower bound of CommitLogPosition owned by this memtable; equal to its predecessor's commitLogUpperBound + private final AtomicReference commitLogLowerBound; + // the write barrier for directing writes to this memtable or the next during a switch + private volatile OpOrder.Barrier writeBarrier; + // the precise upper bound of CommitLogPosition owned by this memtable + private volatile AtomicReference commitLogUpperBound; + + public AbstractMemtableWithCommitlog(TableMetadataRef metadataRef, AtomicReference commitLogLowerBound) + { + super(metadataRef); + this.commitLogLowerBound = commitLogLowerBound; + } + + public CommitLogPosition getApproximateCommitLogLowerBound() + { + return approximateCommitLogLowerBound; + } + + public void switchOut(OpOrder.Barrier writeBarrier, AtomicReference commitLogUpperBound) + { + // This can prepare the memtable data for deletion; it will still be used while the flush is proceeding. + // A setDiscarded call will follow. + assert this.writeBarrier == null; + this.commitLogUpperBound = commitLogUpperBound; + this.writeBarrier = writeBarrier; + } + + public void discard() + { + assert writeBarrier != null : "Memtable must be switched out before being discarded."; + } + + // decide if this memtable should take the write, or if it should go to the next memtable + public boolean accepts(OpOrder.Group opGroup, CommitLogPosition commitLogPosition) + { + // if the barrier hasn't been set yet, then this memtable is still the newest and is taking ALL writes. + OpOrder.Barrier barrier = this.writeBarrier; + if (barrier == null) + return true; + // Note that if this races with taking the barrier the opGroup and commit log position we were given must + // necessarily be before the barrier and any LastCommitLogPosition is set, thus this function will return true + // and no update to commitLogUpperBound is necessary. + + // If the barrier has been set and issued, but is in the past, we are definitely destined for a future memtable. + // Because we issue the barrier after taking LastCommitLogPosition and mutations take their position after + // taking the opGroup, this condition also ensures the given commit log position is greater than the chosen + // upper bound. + if (!barrier.isAfter(opGroup)) + return false; + // We are in the segment of time between the barrier is constructed (and the memtable is switched out) + // and the barrier is issued. + // if we aren't durable we are directed only by the barrier + if (commitLogPosition == null) + return true; + while (true) + { + // If the CL boundary has been set, the mutation can be accepted depending on whether it falls before it. + // However, if it has not been set, the old sstable must still accept writes but we must also ensure that + // their positions are accounted for in the boundary (as there may be a delay between taking the log + // position for the boundary and setting it where a mutation sneaks in). + // Thus, if the boundary hasn't been finalised yet, we simply update it to the max of its current value and + // ours; this permits us to coordinate a safe boundary, as the boundary choice is made atomically wrt our + // max() maintenance, so an operation cannot sneak into the past. + CommitLogPosition currentLast = commitLogUpperBound.get(); + if (currentLast instanceof LastCommitLogPosition) + return currentLast.compareTo(commitLogPosition) >= 0; + if (currentLast != null && currentLast.compareTo(commitLogPosition) >= 0) + return true; + if (commitLogUpperBound.compareAndSet(currentLast, commitLogPosition)) + return true; + } + } + + public CommitLogPosition getCommitLogLowerBound() + { + return commitLogLowerBound.get(); + } + + public LastCommitLogPosition getFinalCommitLogUpperBound() + { + assert commitLogUpperBound != null : "Commit log upper bound should be set before flushing"; + assert commitLogUpperBound.get() instanceof LastCommitLogPosition : "Commit log upper bound has not been sealed yet? " + commitLogUpperBound.get(); + return (LastCommitLogPosition) commitLogUpperBound.get(); + } + + public boolean mayContainDataBefore(CommitLogPosition position) + { + return approximateCommitLogLowerBound.compareTo(position) < 0; + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/DefaultMemtableFactory.java b/src/java/org/apache/cassandra/db/memtable/DefaultMemtableFactory.java new file mode 100644 index 000000000000..d35a21922374 --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/DefaultMemtableFactory.java @@ -0,0 +1,68 @@ +/* + * 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.memtable; + +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.metrics.TableMetrics; +import org.apache.cassandra.schema.TableMetadataRef; + +/** + * This class exists solely to avoid initialization of the default memtable class. + * Some tests want to setup table parameters before initializing DatabaseDescriptor -- this allows them to do so. + */ +public class DefaultMemtableFactory implements Memtable.Factory +{ + @Override + public Memtable create(AtomicReference commitLogLowerBound, TableMetadataRef metadaRef, Memtable.Owner owner) + { + return TrieMemtable.FACTORY.create(commitLogLowerBound, metadaRef, owner); + } + + @Override + public boolean writesShouldSkipCommitLog() + { + return TrieMemtable.FACTORY.writesShouldSkipCommitLog(); + } + + @Override + public boolean writesAreDurable() + { + return TrieMemtable.FACTORY.writesAreDurable(); + } + + @Override + public boolean streamToMemtable() + { + return TrieMemtable.FACTORY.streamToMemtable(); + } + + @Override + public boolean streamFromMemtable() + { + return TrieMemtable.FACTORY.streamFromMemtable(); + } + + @Override + public TableMetrics.ReleasableMetric createMemtableMetrics(TableMetadataRef metadataRef) + { + return TrieMemtable.FACTORY.createMemtableMetrics(metadataRef); + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/Flushing.java b/src/java/org/apache/cassandra/db/memtable/Flushing.java new file mode 100644 index 000000000000..753349fa0d47 --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/Flushing.java @@ -0,0 +1,307 @@ +/* + * 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.memtable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.DiskBoundaries; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.db.commitlog.IntervalSet; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.partitions.AtomicBTreePartition; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.format.SSTableFormat; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.metrics.TableMetrics; +import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.utils.Throwables.maybeFail; + +public class Flushing +{ + private static final Logger logger = LoggerFactory.getLogger(Flushing.class); + + public static List flushRunnables(ColumnFamilyStore cfs, + Memtable memtable, + LifecycleTransaction txn) + { + LifecycleTransaction ongoingFlushTransaction = memtable.setFlushTransaction(txn); + Preconditions.checkState(ongoingFlushTransaction == null, + "Attempted to flush Memtable more than once on %s.%s", + cfs.keyspace.getName(), + cfs.name); + + DiskBoundaries diskBoundaries = cfs.getDiskBoundaries(); + List boundaries = diskBoundaries.getPositions(); + List locations = diskBoundaries.directories; + return flushRunnables(cfs, memtable, boundaries, locations, txn); + } + + @VisibleForTesting + static List flushRunnables(ColumnFamilyStore cfs, + Memtable memtable, + List boundaries, + List locations, + LifecycleTransaction txn) + { + if (boundaries == null) + { + FlushRunnable runnable = flushRunnable(cfs, memtable, null, null, txn, null); + return Collections.singletonList(runnable); + } + + List runnables = new ArrayList<>(boundaries.size()); + PartitionPosition rangeStart = boundaries.get(0).getPartitioner().getMinimumToken().minKeyBound(); + try + { + for (int i = 0; i < boundaries.size(); i++) + { + PartitionPosition t = boundaries.get(i); + FlushRunnable runnable = flushRunnable(cfs, memtable, rangeStart, t, txn, locations.get(i)); + + runnables.add(runnable); + rangeStart = t; + } + return runnables; + } + catch (Throwable e) + { + throw Throwables.propagate(abortRunnables(runnables, e)); + } + } + + @SuppressWarnings("resource") // writer owned by runnable, to be closed or aborted by its caller + static FlushRunnable flushRunnable(ColumnFamilyStore cfs, + Memtable memtable, + PartitionPosition from, + PartitionPosition to, + LifecycleTransaction txn, + Directories.DataDirectory flushLocation) + { + Memtable.FlushCollection flushSet = memtable.getFlushSet(from, to); + SSTableFormat.Type formatType = SSTableFormat.Type.current(); + long estimatedSize = formatType.info.getWriterFactory().estimateSize(flushSet); + + Descriptor descriptor = flushLocation == null + ? cfs.newSSTableDescriptor(cfs.getDirectories().getWriteableLocationAsFile(estimatedSize), formatType) + : cfs.newSSTableDescriptor(cfs.getDirectories().getLocationForDisk(flushLocation), formatType); + + SSTableMultiWriter writer = createFlushWriter(cfs, + flushSet, + txn, + descriptor, + flushSet.partitionCount()); + + return new FlushRunnable(flushSet, writer, cfs.metric, true); + } + + public static Throwable abortRunnables(List runnables, Throwable t) + { + if (runnables != null) + for (FlushRunnable runnable : runnables) + t = runnable.abort(t); + return t; + } + + /** + * The valid states for {@link FlushRunnable} writers. The thread writing the contents + * will transition from IDLE -> RUNNING and back to IDLE when finished using the writer + * or from ABORTING -> ABORTED if another thread has transitioned from RUNNING -> ABORTING. + * We can also transition directly from IDLE -> ABORTED. Whichever threads transitions + * to ABORTED is responsible to abort the writer. + */ + @VisibleForTesting + enum FlushRunnableWriterState + { + IDLE, // the runnable is idle, either not yet started or completed but with the writer waiting to be committed + RUNNING, // the runnable is executing, therefore the writer cannot be aborted or else a SEGV may ensue + ABORTING, // an abort request has been issued, this only happens if abort() is called whilst RUNNING + ABORTED // the writer has been aborted, no resources will be leaked + } + + public static class FlushRunnable implements Callable + { + private final Memtable.FlushCollection toFlush; + + private final SSTableMultiWriter writer; + private final TableMetrics metrics; + private final boolean isBatchLogTable; + private final boolean logCompletion; + private final AtomicReference state; + + public FlushRunnable(Memtable.FlushCollection flushSet, + SSTableMultiWriter writer, + TableMetrics metrics, + boolean logCompletion) + { + this.toFlush = flushSet; + this.writer = writer; + this.metrics = metrics; + this.isBatchLogTable = toFlush.metadata() == SystemKeyspace.Batches; + this.logCompletion = logCompletion; + this.state = new AtomicReference<>(FlushRunnableWriterState.IDLE); + } + + private void writeSortedContents() + { + if (!state.compareAndSet(FlushRunnableWriterState.IDLE, FlushRunnableWriterState.RUNNING)) + { + logger.debug("Failed to write {}, flushed range = ({}, {}], state: {}", + toFlush.memtable().toString(), toFlush.from(), toFlush.to(), state); + return; + } + + long before = System.nanoTime(); + logger.debug("Writing {}, flushed range = ({}, {}], state: {}", + toFlush.memtable().toString(), toFlush.from(), toFlush.to(), state); + + try + { + // (we can't clear out the map as-we-go to free up memory, + // since the memtable is being used for queries in the "pending flush" category) + for (Partition partition : toFlush) + { + if (state.get() == FlushRunnableWriterState.ABORTING) + break; + + // Each batchlog partition is a separate entry in the log. And for an entry, we only do 2 + // operations: 1) we insert the entry and 2) we delete it. Further, BL data is strictly local, + // we don't need to preserve tombstones for repair. So if both operation are in this + // memtable (which will almost always be the case if there is no ongoing failure), we can + // just skip the entry (CASSANDRA-4667). + if (isBatchLogTable && !partition.partitionLevelDeletion().isLive() && partition.hasRows()) + continue; + + if (!partition.isEmpty()) + { + try (UnfilteredRowIterator iter = partition.unfilteredIterator()) + { + writer.append(iter); + } + } + } + } + finally + { + while (true) + { + if (state.compareAndSet(FlushRunnableWriterState.RUNNING, FlushRunnableWriterState.IDLE)) + { + if (logCompletion) + { + long bytesFlushed = writer.getBytesWritten(); + long onDiskBytesWritten = writer.getOnDiskBytesWritten(); + long segmentCount = writer.getSegmentCount(); + logger.info("Completed flushing {} ({}/{} on disk/{} files) for commitlog position {}", + writer.getFilename(), + FBUtilities.prettyPrintMemory(bytesFlushed), + FBUtilities.prettyPrintMemory(onDiskBytesWritten), + segmentCount, + toFlush.memtable().getFinalCommitLogUpperBound()); + // Update the metrics + metrics.incBytesFlushed(toFlush.memtable().getLiveDataSize(), bytesFlushed, before - System.nanoTime()); + metrics.flushSizeOnDisk.update(onDiskBytesWritten); + metrics.flushSegmentCount.update(segmentCount); + } + + break; + } + else if (state.compareAndSet(FlushRunnableWriterState.ABORTING, FlushRunnableWriterState.ABORTED)) + { + logger.debug("Flushing of {} aborted", writer.getFilename()); + maybeFail(writer.abort(null)); + break; + } + } + } + } + + @Override + public SSTableMultiWriter call() + { + writeSortedContents(); + return writer; + // We don't close the writer on error as the caller aborts all runnables if one happens. + } + + public Throwable abort(Throwable throwable) + { + while (true) + { + if (state.compareAndSet(FlushRunnableWriterState.IDLE, FlushRunnableWriterState.ABORTED)) + { + logger.debug("Flushing of {} aborted", writer.getFilename()); + return writer.abort(throwable); + } + else if (state.compareAndSet(FlushRunnableWriterState.RUNNING, FlushRunnableWriterState.ABORTING)) + { + // thread currently executing writeSortedContents() will take care of aborting and throw any exceptions + return throwable; + } + } + } + + @VisibleForTesting + FlushRunnableWriterState state() + { + return state.get(); + } + } + + public static SSTableMultiWriter createFlushWriter(ColumnFamilyStore cfs, + Memtable.FlushCollection flushSet, + LifecycleTransaction txn, + Descriptor descriptor, + long partitionCount) + { + MetadataCollector sstableMetadataCollector = new MetadataCollector(flushSet.metadata().comparator) + .commitLogIntervals(new IntervalSet<>(flushSet.commitLogLowerBound(), + flushSet.commitLogUpperBound())); + + return cfs.createSSTableMultiWriter(descriptor, + partitionCount, + ActiveRepairService.UNREPAIRED_SSTABLE, + ActiveRepairService.NO_PENDING_REPAIR, + false, + sstableMetadataCollector, + new SerializationHeader(true, + flushSet.metadata(), + flushSet.columns(), + flushSet.encodingStats()), + txn); + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/Memtable.java b/src/java/org/apache/cassandra/db/memtable/Memtable.java new file mode 100644 index 000000000000..eb92743089cd --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/Memtable.java @@ -0,0 +1,437 @@ +/* + * 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.memtable; + +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.util.concurrent.ListenableFuture; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.metrics.TableMetrics; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.concurrent.OpOrder; + +/** + * Memtable interface. This defines the operations the ColumnFamilyStore can perform with memtables. + * They are of several types: + * - construction factory interface + * - write and read operations: put, getPartition and makePartitionIterator + * - statistics and features, including partition counts, data size, encoding stats, written columns + * - memory usage tracking, including methods of retrieval and of adding extra allocated space (used non-CFS secondary + * indexes) + * - flush functionality, preparing the set of partitions to flush for given ranges + * - lifecycle management, i.e. operations that prepare and execute switch to a different memtable, together + * with ways of tracking the affected commit log spans + */ +public interface Memtable extends Comparable +{ + // Construction + + /** + * Factory interface for constructing memtables, and querying write durability features. + * + * The factory is chosen using the MemtableParams class (passed as argument to + * {@code CREATE TABLE ... WITH memtable = {...}} or in the memtable options in cassandra.yaml). To make that + * possible, implementations must provide either a static {@code FACTORY} field (if they accept no further option) + * or a static {@code factory(Map)} method. In the latter case, the method should avoid creating + * multiple instances of the factory for the same parameters, or factories should at least implement hashCode and + * equals. + */ + interface Factory + { + /** + * Create a memtable. + * + * @param commitLogLowerBound A commit log lower bound for the new memtable. This will be equal to the previous + * memtable's upper bound and defines the span of positions that any flushed sstable + * will cover. + * @param metadaRef Pointer to the up-to-date table metadata. + * @param owner Owning objects that will receive flush requests triggered by the memtable (e.g. on expiration). + */ + Memtable create(AtomicReference commitLogLowerBound, TableMetadataRef metadaRef, Owner owner); + + /** + * If the memtable can achieve write durability directly (i.e. using some feature other than the commitlog, e.g. + * persistent memory), it can return true here, in which case the commit log will not store mutations in this + * table. + * Note that doing so will prevent point-in-time restores and changed data capture, thus a durable memtable must + * allow the option of turning commit log writing on even if it does not need it. + */ + default boolean writesShouldSkipCommitLog() + { + return false; + } + + /** + * This should be true if the memtable can achieve write durability for crash recovery directly (i.e. using some + * feature other than the commitlog, e.g. persistent memory). + * Setting this flag to true means that the commitlog should not replay mutations for this table on restart, + * and that it should not try to preserve segments that contain relevant data. + * Unless writesShouldSkipCommitLog() is also true, writes will be recorded in the commit log as they may be + * needed for changed data capture or point-in-time restore. + */ + default boolean writesAreDurable() + { + return false; + } + + /** + * Normally we can receive streamed sstables directly, skipping the memtable stage (zero-copy-streaming). When + * the memtable is the primary data store (e.g. persistent memtables), it will usually prefer to receive the + * data instead. + * + * If this returns true, all streamed sstables's content will be read and replayed as mutations, disabling + * zero-copy streaming. + */ + default boolean streamToMemtable() + { + return false; + } + + /** + * When we need to stream data, we usually flush and stream the resulting sstables. This will not work correctly + * if the memtable does not want to flush for streaming (e.g. persistent memtables acting as primary data + * store), because data (not just recent) will be missing from the streamed view. Such memtables must present + * their data separately for streaming. + * In other words if the memtable returns false on shouldSwitch(STREAMING/REPAIR), its factory must return true + * here. + * + * If this flag returns true, streaming will write the relevant content that resides in the memtable to + * temporary sstables, stream these sstables and then delete them. + */ + default boolean streamFromMemtable() + { + return false; + } + + /** + * Memtable metrics lifecycle matches table lifecycle. It is the table + * that owns the metrics and decides when to release them; + */ + default TableMetrics.ReleasableMetric createMemtableMetrics(TableMetadataRef metadataRef) + { + return null; + } + } + + /** + * Interface for providing signals back to the owner. + */ + interface Owner + { + /** Signal to the owner that a flush is required (e.g. in response to hitting space limits) */ + ListenableFuture signalFlushRequired(Memtable memtable, ColumnFamilyStore.FlushReason reason); + + /** Get the current memtable for this owner. Used to avoid capturing memtable in scheduled flush tasks. */ + Memtable getCurrentMemtable(); + + /** + * Collect the index memtables flushed together with this. Used to accurately calculate memory that would be + * freed by a flush. + */ + Iterable getIndexMemtables(); + + ShardBoundaries localRangeSplits(int shardCount); + } + + // Main write and read operations + + /** + * Put new data in the memtable. This operation may block until enough memory is available in the memory pool. + * + * @param update the partition update, may be a new partition or an update to an existing one + * @param indexer receives information about the update's effect + * @param opGroup write operation group, used to permit the operation to complete if it is needed to complete a + * flush to free space. + * + * @return the smallest timestamp delta between corresponding rows from existing and update. A + * timestamp delta being computed as the difference between the cells and DeletionTimes from any existing partition + * and those in {@code update}. See CASSANDRA-7979. + */ + long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup); + + /** + * Get the partition for the specified key. Returns null if no such partition is present. + */ + Partition getPartition(DecoratedKey key); + + /** + * Returns a partition iterator for the given data range. + * + * @param columnFilter filter to apply to all returned partitions + * @param dataRange the partition and clustering range queried + */ + MemtableUnfilteredPartitionIterator makePartitionIterator(ColumnFilter columnFilter, + DataRange dataRange); + + interface MemtableUnfilteredPartitionIterator extends UnfilteredPartitionIterator + { + /** + * Returns the minimum local deletion time for all partitions in the range. + * Required for the efficiency of partition range read commands. + */ + int getMinLocalDeletionTime(); + } + + + // Statistics + + /** Number of partitions stored in the memtable */ + long partitionCount(); + + /** Size of the data not accounting for any metadata / mapping overheads */ + long getLiveDataSize(); + + /** + * Number of "operations" (in the sense defined in {@link PartitionUpdate#operationCount()}) the memtable has + * executed. + */ + long getOperations(); + + /** Minimum timestamp of all stored data */ + long getMinTimestamp(); + + /** + * The table's definition metadata. + * + * Note that this tracks the current state of the table and is not necessarily the same as what was used to create + * the memtable. + */ + TableMetadata metadata(); + + + // Memory usage tracking + + /** + * Add this memtable's used memory to the given usage object. This can be used to retrieve a single memtable's usage + * as well as to combine the ones of related sstables (e.g. a table and its table-based secondary indexes). + */ + void addMemoryUsageTo(MemoryUsage usage); + + + /** + * Creates a holder for memory usage collection. + * + * This is used to track on- and off-heap memory, as well as the ratio to the total permitted memtable memory. + */ + static MemoryUsage newMemoryUsage() + { + return new MemoryUsage(); + } + + /** + * Shorthand for the getting a given table's memory usage. + * Implemented as a static to prevent implementations altering expectations by e.g. returning a cached object. + */ + static MemoryUsage getMemoryUsage(Memtable memtable) + { + MemoryUsage usage = newMemoryUsage(); + memtable.addMemoryUsageTo(usage); + return usage; + } + + class MemoryUsage + { + /** On-heap memory used in bytes */ + public long ownsOnHeap = 0; + /** Off-heap memory used in bytes */ + public long ownsOffHeap = 0; + /** On-heap memory as ratio to permitted memtable space */ + public float ownershipRatioOnHeap = 0.0f; + /** Off-heap memory as ratio to permitted memtable space */ + public float ownershipRatioOffHeap = 0.0f; + + public String toString() + { + return String.format("%s (%.0f%%) on-heap, %s (%.0f%%) off-heap", + FBUtilities.prettyPrintMemory(ownsOnHeap), + ownershipRatioOnHeap * 100, + FBUtilities.prettyPrintMemory(ownsOffHeap), + ownershipRatioOffHeap * 100); + } + } + + /** + * Adjust the used on-heap space by the given size (e.g. to reflect memory used by a non-table-based index). + * This operation may block until enough memory is available in the memory pool. + * + * @param additionalSpace the number of allocated bytes + * @param opGroup write operation group, used to permit the operation to complete if it is needed to complete a + * flush to free space. + */ + void markExtraOnHeapUsed(long additionalSpace, OpOrder.Group opGroup); + + /** + * Adjust the used off-heap space by the given size (e.g. to reflect memory used by a non-table-based index). + * This operation may block until enough memory is available in the memory pool. + * + * @param additionalSpace the number of allocated bytes + * @param opGroup write operation group, used to permit the operation to complete if it is needed to complete a + * flush to free space. + */ + void markExtraOffHeapUsed(long additionalSpace, OpOrder.Group opGroup); + + + // Flushing + + /** + * Get the collection of data between the given partition boundaries in a form suitable for flushing. + */ + FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to); + + /** + * A collection of partitions for flushing plus some information required for writing an sstable. + * + * Note that the listed entries must conform with the specified metadata. In particular, if the memtable is still + * being written to, care must be taken to not list newer items as they may violate the bounds collected by the + * encoding stats or refer to columns that don't exist in the collected columns set. + */ + interface FlushCollection

    extends Iterable

    , SSTableWriter.SSTableSizeParameters + { + Memtable memtable(); + + PartitionPosition from(); + PartitionPosition to(); + + /** The commit log position at the time that this memtable was created */ + CommitLogPosition commitLogLowerBound(); + /** The commit log position at the time that this memtable was switched out */ + CommitLogPosition commitLogUpperBound(); + + /** The set of all columns that have been written */ + RegularAndStaticColumns columns(); + /** Statistics required for writing an sstable efficiently */ + EncodingStats encodingStats(); + + default TableMetadata metadata() + { + return memtable().metadata(); + } + + long partitionCount(); + default boolean isEmpty() + { + return partitionCount() > 0; + } + } + + + // Lifecycle management + + /** + * Called to tell the memtable that it is being switched out and will be flushed (or dropped) and discarded. + * Will be followed by a {@link #getFlushSet} call (if the table is not truncated or dropped), and a + * {@link #discard}. + * + * @param writeBarrier The barrier that will signal that all writes to this memtable have completed. That is, the + * point after which writes cannot be accepted by this memtable (it is permitted for writes + * before this barrier to go into the next; see {@link #accepts}). + * @param commitLogUpperBound The upper commit log position for this memtable. The value may be modified after this + * call and will match the next memtable's lower commit log bound. + */ + void switchOut(OpOrder.Barrier writeBarrier, AtomicReference commitLogUpperBound); + + /** + * This memtable is no longer in use or required for outstanding flushes or operations. + * All held memory must be released. + */ + void discard(); + + /** + * Decide if this memtable should take a write with the given parameters, or if the write should go to the next + * memtable. This enforces that no writes after the barrier set by {@link #switchOut} can be accepted, and + * is also used to define a shared commit log bound as the upper for this memtable and lower for the next. + */ + boolean accepts(OpOrder.Group opGroup, CommitLogPosition commitLogPosition); + + /** Approximate commit log lower bound, <= getCommitLogLowerBound, used as a time stamp for ordering */ + CommitLogPosition getApproximateCommitLogLowerBound(); + + /** The commit log position at the time that this memtable was created */ + CommitLogPosition getCommitLogLowerBound(); + + /** The commit log position at the time that this memtable was switched out */ + LastCommitLogPosition getFinalCommitLogUpperBound(); + + /** True if the memtable can contain any data that was written before the given commit log position */ + boolean mayContainDataBefore(CommitLogPosition position); + + /** True if the memtable contains no data */ + boolean isClean(); + + /** These two methods provide a way of tracking on-going flushes */ + public LifecycleTransaction setFlushTransaction(LifecycleTransaction transaction); + public LifecycleTransaction getFlushTransaction(); + + /** Order memtables by time as reflected in the commit log position at time of construction */ + default int compareTo(Memtable that) + { + return this.getApproximateCommitLogLowerBound().compareTo(that.getApproximateCommitLogLowerBound()); + } + + /** + * Decides whether the memtable should be switched/flushed for the passed reason. + * Normally this will return true, but e.g. persistent memtables may choose not to flush. Returning false will + * trigger further action for some reasons: + * - SCHEMA_CHANGE will be followed by metadataUpdated(). + * - SNAPSHOT will be followed by performSnapshot(). + * - STREAMING/REPAIR will be followed by creating a FlushSet for the streamed/repaired ranges. This data will be + * used to create sstables, which will be streamed and then deleted. + * This will not be called if the sstable is switched because of truncation or drop. + */ + boolean shouldSwitch(ColumnFamilyStore.FlushReason reason); + + /** + * Called when the table's metadata is updated. The memtable's metadata reference now points to the new version. + */ + void metadataUpdated(); + + /** + * If the memtable needs to do some special action for snapshots (e.g. because it is persistent and does not want + * to flush), it should return false on the above with reason SNAPSHOT and implement this method. + */ + void performSnapshot(String snapshotName); + + /** + * Special commit log position marker used in the upper bound marker setting process + * (see {@link org.apache.cassandra.db.ColumnFamilyStore#setCommitLogUpperBound} and {@link AbstractMemtable#accepts}) + */ + public static final class LastCommitLogPosition extends CommitLogPosition + { + public LastCommitLogPosition(CommitLogPosition copy) + { + super(copy.segmentId, copy.position); + } + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/PersistentMemoryMemtable.java b/src/java/org/apache/cassandra/db/memtable/PersistentMemoryMemtable.java new file mode 100644 index 000000000000..c44f1bd12449 --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/PersistentMemoryMemtable.java @@ -0,0 +1,261 @@ +/* + * 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.memtable; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.concurrent.OpOrder; + +/** + * Skeleton for persistent memory memtable. + */ +public class PersistentMemoryMemtable +//extends AbstractMemtable +extends SkipListMemtable // to test framework +{ + public PersistentMemoryMemtable(TableMetadataRef metadaRef, Owner owner) + { + super(null, metadaRef, owner); + // We should possibly link the persistent data of this memtable + } + + public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) + { + // TODO: implement + return super.put(update, indexer, opGroup); + } + + public MemtableUnfilteredPartitionIterator makePartitionIterator(ColumnFilter columnFilter, DataRange dataRange) + { + // TODO: implement + return super.makePartitionIterator(columnFilter, dataRange); + } + + public Partition getPartition(DecoratedKey key) + { + // TODO: implement + return super.getPartition(key); + } + + public long partitionCount() + { + // TODO: implement + return super.partitionCount(); + } + + public FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to) + { + // TODO: implement + // FIXME: If the memtable can still be written to, this uses a view of the metadata that may not be up-to-date + // with the content. This may cause streaming to fail e.g. if a new column appears and is added to some row in + // the memtable between the time that this is constructed and the relevant row is written. Such failures should + // be recoverable by redoing the stream. + // If an implementation can produce a view/snapshot of the data at a point before the features were collected, + // this problem will not occur. + return super.getFlushSet(from, to); + } + + public boolean shouldSwitch(ColumnFamilyStore.FlushReason reason) + { + // We want to avoid all flushing. + switch (reason) + { + case STARTUP: // Called after reading and replaying the commit log. + case SHUTDOWN: // Called to flush data before shutdown. + case INTERNALLY_FORCED: // Called to ensure ordering and persistence of system table events. + case MEMTABLE_PERIOD_EXPIRED: // The specified memtable expiration time elapsed. + case INDEX_TABLE_FLUSH: // Flush requested on index table because main table is flushing. + case STREAMS_RECEIVED: // Flush to save streamed data that was written to memtable. + return false; // do not do anything + + case INDEX_BUILD_COMPLETED: + case INDEX_REMOVED: + // Both of these are needed as safepoints for index management. Nothing to do. + return false; + + case VIEW_BUILD_STARTED: + case INDEX_BUILD_STARTED: + // TODO: Figure out secondary indexes and views. + return false; + + case SCHEMA_CHANGE: + if (!(metadata().params.memtable.factory instanceof Factory)) + return true; // User has switched to a different memtable class. Flush and release all held data. + // Otherwise, assuming we can handle the change, don't switch. + // TODO: Handle + return false; + + case STREAMING: // Called to flush data so it can be streamed. TODO: How dow we stream? + case REPAIR: // Called to flush data for repair. TODO: How do we repair? + // ColumnFamilyStore will create sstables of the affected ranges which will not be consulted on reads and + // will be deleted after streaming. + return false; + + case SNAPSHOT: + // We don't flush for this. Returning false will trigger a performSnapshot call. + return false; + + case DROP: // Called when a table is dropped. This memtable is no longer necessary. + case TRUNCATE: // The data is being deleted, but the table remains. + // Returning true asks the ColumnFamilyStore to replace this memtable object without flushing. + // This will call discard() below to delete all held data. + return true; + + case MEMTABLE_LIMIT: // The memtable size limit is reached, and this table was selected for flushing. + // Also passed if we call owner.signalLimitReached() + case COMMITLOG_DIRTY: // Commitlog thinks it needs to keep data from this table. + // Neither of the above should happen as we specify writesAreDurable and don't use an allocator/cleaner. + throw new AssertionError(); + + case USER_FORCED: + case UNIT_TESTS: + return false; + default: + throw new AssertionError(); + } + } + + public void metadataUpdated() + { + // TODO: handle + } + + public void performSnapshot(String snapshotName) + { + // TODO: implement. Figure out how to restore snapshot (with external tools). + } + + public void switchOut(OpOrder.Barrier writeBarrier, AtomicReference commitLogUpperBound) + { + super.switchOut(writeBarrier, commitLogUpperBound); + // This can prepare the memtable data for deletion; it will still be used while the flush is proceeding. + // A discard call will follow. + } + + public void discard() + { + // This will be called to release/delete all held data because the memtable is switched, due to having + // its data flushed, due to a truncate/drop, or due to a schema change to a different memtable class. + + // TODO: Implement. This should delete all memtable data from pmem. + super.discard(); + } + + public CommitLogPosition getApproximateCommitLogLowerBound() + { + // We don't maintain commit log positions + return CommitLogPosition.NONE; + } + + public CommitLogPosition getCommitLogLowerBound() + { + // We don't maintain commit log positions + return CommitLogPosition.NONE; + } + + public LastCommitLogPosition getFinalCommitLogUpperBound() + { + // We don't maintain commit log positions + return new LastCommitLogPosition(CommitLogPosition.NONE); + } + + public boolean isClean() + { + return partitionCount() == 0; + } + + public boolean mayContainDataBefore(CommitLogPosition position) + { + // We don't track commit log positions, so if we are dirty, we may. + return !isClean(); + } + + public void addMemoryUsageTo(MemoryUsage stats) + { + // our memory usage is not counted + } + + public void markExtraOnHeapUsed(long additionalSpace, OpOrder.Group opGroup) + { + // we don't track this + } + + public void markExtraOffHeapUsed(long additionalSpace, OpOrder.Group opGroup) + { + // we don't track this + } + + public static Factory factory(Map furtherOptions) + { + Boolean skipOption = Boolean.parseBoolean(furtherOptions.remove("skipCommitLog")); + return skipOption ? commitLogSkippingFactory : commitLogWritingFactory; + } + + private static final Factory commitLogSkippingFactory = new Factory(true); + private static final Factory commitLogWritingFactory = new Factory(false); + + static class Factory implements Memtable.Factory + { + private final boolean skipCommitLog; + + public Factory(boolean skipCommitLog) + { + this.skipCommitLog = skipCommitLog; + } + + public Memtable create(AtomicReference commitLogLowerBound, + TableMetadataRef metadaRef, + Owner owner) + { + return new PersistentMemoryMemtable(metadaRef, owner); + } + + public boolean writesShouldSkipCommitLog() + { + return skipCommitLog; + } + + public boolean writesAreDurable() + { + return true; + } + + public boolean streamToMemtable() + { + return true; + } + + public boolean streamFromMemtable() + { + return true; + } + } + +} diff --git a/src/java/org/apache/cassandra/db/memtable/ShardBoundaries.java b/src/java/org/apache/cassandra/db/memtable/ShardBoundaries.java new file mode 100644 index 000000000000..720dae33d94f --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/ShardBoundaries.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.db.memtable; + +import java.util.Arrays; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.dht.Token; + +/** + * Holds boundaries (tokens) used to map a particular token (so partition key) to a shard id. + * In practice, each keyspace has its associated boundaries, see {@link Keyspace}. + *

    + * Technically, if we use {@code n} shards, this is a list of {@code n-1} tokens and each token {@code tk} gets assigned + * to the core ID corresponding to the slot of the smallest token in the list that is greater to {@code tk}, or {@code n} + * if {@code tk} is bigger than any token in the list. + */ +public class ShardBoundaries +{ + private static final PartitionPosition[] EMPTY_TOKEN_ARRAY = new PartitionPosition[0]; + + // Special boundaries that map all tokens to one shard. + // These boundaries will be used in either of these cases: + // - there is only 1 shard configured + // - the default partitioner doesn't support splitting + // - the keyspace is local system keyspace + public static final ShardBoundaries NONE = new ShardBoundaries(EMPTY_TOKEN_ARRAY, -1); + + private final PartitionPosition[] boundaries; + public final long ringVersion; + + @VisibleForTesting + public ShardBoundaries(PartitionPosition[] boundaries, long ringVersion) + { + this.boundaries = boundaries; + this.ringVersion = ringVersion; + } + + public ShardBoundaries(List boundaries, long ringVersion) + { + this(boundaries.toArray(EMPTY_TOKEN_ARRAY), ringVersion); + } + + /** + * Computes the shard to use for the provided token. + */ + public int getShardForToken(Token tk) + { + for (int i = 0; i < boundaries.length; i++) + { + if (tk.compareTo(boundaries[i].getToken()) < 0) + return i; + } + return boundaries.length; + } + + /** + * Computes the shard to use for the provided key. + */ + public int getShardForKey(DecoratedKey key) + { + // Boundaries are missing if the node is not sufficiently initialized yet + if (boundaries.length == 0) + return 0; + + assert (key.getPartitioner() == DatabaseDescriptor.getPartitioner()); + return getShardForToken(key.getToken()); + } + + /** + * The number of shards that this boundaries support, that is how many different shard ids {@link #getShardForToken} might + * possibly return. + * + * @return the number of shards supported by theses boundaries. + */ + public int shardCount() + { + return boundaries.length + 1; + } + + @Override + public String toString() + { + if (boundaries.length == 0) + return "shard 0: (min, max)"; + + StringBuilder sb = new StringBuilder(); + sb.append("shard 0: (min, ").append(boundaries[0]).append(") "); + for (int i = 0; i < boundaries.length - 1; i++) + sb.append("shard ").append(i+1).append(": (").append(boundaries[i]).append(", ").append(boundaries[i+1]).append("] "); + sb.append("shard ").append(boundaries.length).append(": (").append(boundaries[boundaries.length-1]).append(", max)"); + return sb.toString(); + } + + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ShardBoundaries that = (ShardBoundaries) o; + + return Arrays.equals(boundaries, that.boundaries); + } + + public int hashCode() + { + return Arrays.hashCode(boundaries); + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/SkipListMemtable.java b/src/java/org/apache/cassandra/db/memtable/SkipListMemtable.java new file mode 100644 index 000000000000..1255716aecfb --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/SkipListMemtable.java @@ -0,0 +1,369 @@ +/* + * 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.memtable; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.ListenableFuture; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.BufferDecoratedKey; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.AtomicBTreePartition; +import org.apache.cassandra.db.partitions.BTreePartitionData; +import org.apache.cassandra.db.partitions.BTreePartitionUpdater; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.IncludingExcludingBounds; +import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.concurrent.OpOrder; +import org.apache.cassandra.utils.memory.MemtableAllocator; + +public class SkipListMemtable extends AbstractAllocatorMemtable +{ + private static final Logger logger = LoggerFactory.getLogger(SkipListMemtable.class); + + public static final Factory FACTORY = SkipListMemtable::new; + + private static final int ROW_OVERHEAD_HEAP_SIZE = estimateRowOverhead(Integer.parseInt(System.getProperty("cassandra.memtable_row_overhead_computation_step", "100000"))); + + // We index the memtable by PartitionPosition only for the purpose of being able + // to select key range using Token.KeyBound. However put() ensures that we + // actually only store DecoratedKey. + private final ConcurrentNavigableMap partitions = new ConcurrentSkipListMap<>(); + + private final AtomicLong liveDataSize = new AtomicLong(0); + + SkipListMemtable(AtomicReference commitLogLowerBound, TableMetadataRef metadataRef, Owner owner) + { + super(commitLogLowerBound, metadataRef, owner); + } + + // Only for testing + @VisibleForTesting + public SkipListMemtable(TableMetadataRef metadataRef) + { + this(null, metadataRef, new Owner() + { + @Override + public ListenableFuture signalFlushRequired(Memtable memtable, ColumnFamilyStore.FlushReason reason) + { + return null; + } + + @Override + public Memtable getCurrentMemtable() + { + return null; + } + + @Override + public Iterable getIndexMemtables() + { + return Collections.emptyList(); + } + + public ShardBoundaries localRangeSplits(int shardCount) + { + return null; // not implemented + } + }); + } + + protected Factory factory() + { + return FACTORY; + } + + @Override + public void addMemoryUsageTo(MemoryUsage stats) + { + super.addMemoryUsageTo(stats); + } + + public boolean isClean() + { + return partitions.isEmpty(); + } + + /** + * Should only be called by ColumnFamilyStore.apply via Keyspace.apply, which supplies the appropriate + * OpOrdering. + * + * commitLogSegmentPosition should only be null if this is a secondary index, in which case it is *expected* to be null + */ + public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) + { + AtomicBTreePartition previous = partitions.get(update.partitionKey()); + + long initialSize = 0; + if (previous == null) + { + final DecoratedKey cloneKey = allocator.clone(update.partitionKey(), opGroup); + AtomicBTreePartition empty = new AtomicBTreePartition(metadata, cloneKey, allocator); + // We'll add the columns later. This avoids wasting works if we get beaten in the putIfAbsent + previous = partitions.putIfAbsent(cloneKey, empty); + if (previous == null) + { + previous = empty; + // allocate the row overhead after the fact; this saves over allocating and having to free after, but + // means we can overshoot our declared limit. + int overhead = (int) (cloneKey.getToken().getHeapSize() + ROW_OVERHEAD_HEAP_SIZE); + allocator.onHeap().allocate(overhead, opGroup); + initialSize = 8; + } + } + + BTreePartitionUpdater updater = previous.addAll(update, opGroup, indexer); + updateMin(minTimestamp, previous.stats().minTimestamp); + liveDataSize.addAndGet(initialSize + updater.dataSize); + columnsCollector.update(update.columns()); + statsCollector.update(update.stats()); + currentOperations.addAndGet(update.operationCount()); + return updater.colUpdateTimeDelta; + } + + public long partitionCount() + { + return partitions.size(); + } + + public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFilter columnFilter, + final DataRange dataRange) + { + AbstractBounds keyRange = dataRange.keyRange(); + + PartitionPosition left = keyRange.left; + PartitionPosition right = keyRange.right; + + boolean isBound = keyRange instanceof Bounds; + boolean includeLeft = isBound || keyRange instanceof IncludingExcludingBounds; + boolean includeRight = isBound || keyRange instanceof Range; + Map subMap = getPartitionsSubMap(left, + includeLeft, + right, + includeRight); + + return new MemtableUnfilteredPartitionIterator(metadata.get(), subMap, columnFilter, dataRange); + } + + private Map getPartitionsSubMap(PartitionPosition left, + boolean includeLeft, + PartitionPosition right, + boolean includeRight) + { + if (left != null && left.isMinimum()) + left = null; + if (right != null && right.isMinimum()) + right = null; + + try + { + if (left == null) + return right == null ? partitions : partitions.headMap(right, includeRight); + else + return right == null + ? partitions.tailMap(left, includeLeft) + : partitions.subMap(left, includeLeft, right, includeRight); + } + catch (IllegalArgumentException e) + { + logger.error("Invalid range requested {} - {}", left, right); + throw e; + } + } + + public Partition getPartition(DecoratedKey key) + { + return partitions.get(key); + } + + private static int estimateRowOverhead(final int count) + { + // calculate row overhead + try (final OpOrder.Group group = new OpOrder().start()) + { + int rowOverhead; + MemtableAllocator allocator = MEMORY_POOL.newAllocator(); + ConcurrentNavigableMap partitions = new ConcurrentSkipListMap<>(); + final Object val = new Object(); + for (int i = 0 ; i < count ; i++) + partitions.put(allocator.clone(new BufferDecoratedKey(new LongToken(i), ByteBufferUtil.EMPTY_BYTE_BUFFER), group), val); + double avgSize = ObjectSizes.measureDeep(partitions) / (double) count; + rowOverhead = (int) ((avgSize - Math.floor(avgSize)) < 0.05 ? Math.floor(avgSize) : Math.ceil(avgSize)); + rowOverhead -= ObjectSizes.measureDeep(new LongToken(0)); + rowOverhead += AtomicBTreePartition.EMPTY_SIZE; + rowOverhead += BTreePartitionData.UNSHARED_HEAP_SIZE; + allocator.setDiscarding(); + allocator.setDiscarded(); + return rowOverhead; + } + } + + public FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to) + { + Map toFlush = getPartitionsSubMap(from, true, to, false); + long keySize = 0; + + boolean trackContention = logger.isTraceEnabled(); + if (trackContention) + { + int heavilyContendedRowCount = 0; + + for (AtomicBTreePartition partition : toFlush.values()) + { + keySize += partition.partitionKey().getKey().remaining(); + if (trackContention && partition.useLock()) + heavilyContendedRowCount++; + } + + if (heavilyContendedRowCount > 0) + logger.trace("High update contention in {}/{} partitions of {} ", heavilyContendedRowCount, toFlush.size(), SkipListMemtable.this); + } + else + { + for (PartitionPosition key : toFlush.keySet()) + { + // make sure we don't write non-sensical keys + assert key instanceof DecoratedKey; + keySize += ((DecoratedKey) key).getKey().remaining(); + } + } + final long partitionKeySize = keySize; + + return new AbstractFlushCollection() + { + public Memtable memtable() + { + return SkipListMemtable.this; + } + + public PartitionPosition from() + { + return from; + } + + public PartitionPosition to() + { + return to; + } + + public long partitionCount() + { + return toFlush.size(); + } + + public Iterator iterator() + { + return toFlush.values().iterator(); + } + + public long partitionKeySize() + { + return partitionKeySize; + } + }; + } + + + public static class MemtableUnfilteredPartitionIterator extends AbstractUnfilteredPartitionIterator implements Memtable.MemtableUnfilteredPartitionIterator + { + private final TableMetadata metadata; + private final Iterator> iter; + private final Map source; + private final ColumnFilter columnFilter; + private final DataRange dataRange; + + public MemtableUnfilteredPartitionIterator(TableMetadata metadata, Map map, ColumnFilter columnFilter, DataRange dataRange) + { + this.metadata = metadata; + this.source = map; + this.iter = map.entrySet().iterator(); + this.columnFilter = columnFilter; + this.dataRange = dataRange; + } + + public int getMinLocalDeletionTime() + { + int minLocalDeletionTime = Integer.MAX_VALUE; + for (AtomicBTreePartition partition : source.values()) + minLocalDeletionTime = Math.min(minLocalDeletionTime, partition.stats().minLocalDeletionTime); + + return minLocalDeletionTime; + } + + public TableMetadata metadata() + { + return metadata; + } + + public boolean hasNext() + { + return iter.hasNext(); + } + + public UnfilteredRowIterator next() + { + Map.Entry entry = iter.next(); + // Actual stored key should be true DecoratedKey + assert entry.getKey() instanceof DecoratedKey; + DecoratedKey key = (DecoratedKey)entry.getKey(); + ClusteringIndexFilter filter = dataRange.clusteringIndexFilter(key); + + return filter.getUnfilteredRowIterator(columnFilter, entry.getValue()); + } + } + + public long getLiveDataSize() + { + return liveDataSize.get(); + } + + /** + * For testing only. Give this memtable too big a size to make it always fail flushing. + */ + @VisibleForTesting + public void makeUnflushable() + { + liveDataSize.addAndGet(1024L * 1024 * 1024 * 1024 * 1024); + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java new file mode 100644 index 000000000000..8520d15c7daf --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -0,0 +1,739 @@ +/* + * 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.memtable; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterators; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.BufferDecoratedKey; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.commitlog.CommitLogPosition; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.BTreePartitionData; +import org.apache.cassandra.db.partitions.BTreePartitionUpdater; +import org.apache.cassandra.db.partitions.ImmutableBTreePartition; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.tries.MemtableTrie; +import org.apache.cassandra.db.tries.Trie; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.dht.IncludingExcludingBounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.metrics.TableMetrics; +import org.apache.cassandra.metrics.TrieMemtableMetricsView; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.concurrent.OpOrder; +import org.apache.cassandra.utils.memory.EnsureOnHeap; +import org.apache.cassandra.utils.memory.MemtableAllocator; + +public class TrieMemtable extends AbstractAllocatorMemtable +{ + private static final Logger logger = LoggerFactory.getLogger(TrieMemtable.class); + public static final String TRIE_MEMTABLE_CONFIG_OBJECT_NAME = "org.apache.cassandra.db:type=TrieMemtableConfig"; + + public static final Factory FACTORY = new TrieMemtable.Factory(); + + /** Buffer type to use for memtable tries (on- vs off-heap) */ + public static final BufferType BUFFER_TYPE; + + static + { + switch (DatabaseDescriptor.getMemtableAllocationType()) + { + case unslabbed_heap_buffers: + case heap_buffers: + BUFFER_TYPE = BufferType.ON_HEAP; + break; + case offheap_buffers: + case offheap_objects: + BUFFER_TYPE = BufferType.OFF_HEAP; + break; + default: + throw new AssertionError(); + } + + MBeanWrapper.instance.registerMBean(new TrieMemtableConfig(), TRIE_MEMTABLE_CONFIG_OBJECT_NAME, MBeanWrapper.OnException.LOG); + } + + /** If keys is below this length, we will use a recursive procedure for inserting data in the memtable trie. */ + @VisibleForTesting + public static final int MAX_RECURSIVE_KEY_LENGTH = 128; + + /** The byte-ordering conversion version to use for memtables. */ + public static final ByteComparable.Version BYTE_COMPARABLE_VERSION = ByteComparable.Version.OSS41; + + // Set to true when the memtable requests a switch (e.g. for trie size limit being reached) to ensure only one + // thread calls cfs.switchMemtableIfCurrent. + private AtomicBoolean switchRequested = new AtomicBoolean(false); + + + /** + * Core-specific memtable regions. All writes must go through the specific core. The data structures used + * are concurrent-read safe, thus reads can be carried out from any thread. + */ + private final MemtableShard[] shards; + + /** + * A merged view of the memtable map. Used for partition range queries and flush. + * For efficiency we serve single partition requests off the shard which offers more direct MemtableTrie methods. + */ + private final Trie mergedTrie; + + private final TrieMemtableMetricsView metrics; + + @VisibleForTesting + public static final String SHARD_COUNT_PROPERTY = "cassandra.trie.memtable.shard.count"; + + private static volatile int SHARD_COUNT = Integer.getInteger(SHARD_COUNT_PROPERTY, FBUtilities.getAvailableProcessors()); + + // only to be used by init(), to setup the very first memtable for the cfs + TrieMemtable(AtomicReference commitLogLowerBound, TableMetadataRef metadataRef, Owner owner) + { + super(commitLogLowerBound, metadataRef, owner); + this.metrics = TrieMemtableMetricsView.getOrCreate(metadataRef.keyspace, metadataRef.name); + this.shards = generatePartitionShards(SHARD_COUNT, metadataRef, metrics); + this.mergedTrie = makeMergedTrie(shards); + logger.debug("Created memtable with {} shards", this.shards.length); + } + + private static MemtableShard[] generatePartitionShards(int splits, + TableMetadataRef metadata, + TrieMemtableMetricsView metrics) + { + if (splits == 1) + return new MemtableShard[] { new MemtableShard(0, metadata, metrics) }; + + MemtableShard[] partitionMapContainer = new MemtableShard[splits]; + for (int i = 0; i < splits; i++) + partitionMapContainer[i] = new MemtableShard(i, metadata, metrics); + + return partitionMapContainer; + } + + private static Trie makeMergedTrie(MemtableShard[] shards) + { + List> tries = new ArrayList<>(shards.length); + for (MemtableShard shard : shards) + tries.add(shard.data); + return Trie.mergeDistinct(tries); + } + + protected Factory factory() + { + return FACTORY; + } + + public boolean isClean() + { + for (MemtableShard shard : shards) + if (!shard.isEmpty()) + return false; + return true; + } + + @VisibleForTesting + @Override + public void switchOut(OpOrder.Barrier writeBarrier, AtomicReference commitLogUpperBound) + { + super.switchOut(writeBarrier, commitLogUpperBound); + + for (MemtableShard shard : shards) + shard.allocator.setDiscarding(); + } + + @Override + public void discard() + { + super.discard(); + // metrics here are not thread safe, but I think we can live with that + metrics.lastFlushShardDataSizes.reset(); + for (MemtableShard shard : shards) + { + metrics.lastFlushShardDataSizes.update(shard.liveDataSize()); + } + for (MemtableShard shard : shards) + { + shard.allocator.setDiscarded(); + shard.data.discardBuffers(); + } + } + + int getShardForKey(DecoratedKey key) + { + return Math.floorMod(key.filterHashLowerBits(), SHARD_COUNT); + } + + /** + * Should only be called by ColumnFamilyStore.apply via Keyspace.apply, which supplies the appropriate + * OpOrdering. + * + * commitLogSegmentPosition should only be null if this is a secondary index, in which case it is *expected* to be null + */ + public long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) + { + DecoratedKey key = update.partitionKey(); + MemtableShard shard = shards[getShardForKey(key)]; + long colUpdateTimeDelta = shard.put(key, update, indexer, opGroup); + + if (shard.data.reachedAllocatedSizeThreshold() && !switchRequested.getAndSet(true)) + { + logger.info("Scheduling flush due to trie size limit reached."); + owner.signalFlushRequired(this, ColumnFamilyStore.FlushReason.MEMTABLE_LIMIT); + } + + return colUpdateTimeDelta; + } + + @Override + public void addMemoryUsageTo(MemoryUsage stats) + { + super.addMemoryUsageTo(stats); + for (MemtableShard shard : shards) + { + stats.ownsOnHeap += shard.allocator.onHeap().owns(); + stats.ownsOffHeap += shard.allocator.offHeap().owns(); + stats.ownershipRatioOnHeap += shard.allocator.onHeap().ownershipRatio(); + stats.ownershipRatioOffHeap += shard.allocator.offHeap().ownershipRatio(); + } + } + + /** + * Technically we should scatter gather on all the core threads because the size in following calls are not + * using volatile variables, but for metrics purpose this should be good enough. + */ + @Override + public long getLiveDataSize() + { + long total = 0L; + for (MemtableShard shard : shards) + total += shard.liveDataSize(); + return total; + } + + @Override + public long getOperations() + { + long total = 0L; + for (MemtableShard shard : shards) + total += shard.currentOperations(); + return total; + } + + @Override + public long partitionCount() + { + int total = 0; + for (MemtableShard shard : shards) + total += shard.size(); + return total; + } + + @Override + public long getMinTimestamp() + { + long min = Long.MAX_VALUE; + for (MemtableShard shard : shards) + min = Long.min(min, shard.minTimestamp()); + return min; + } + + @Override + RegularAndStaticColumns columns() + { + for (MemtableShard shard : shards) + columnsCollector.update(shard.columnsCollector); + return columnsCollector.get(); + } + + @Override + EncodingStats encodingStats() + { + for (MemtableShard shard : shards) + statsCollector.update(shard.statsCollector.get()); + return statsCollector.get(); + } + + public MemtableUnfilteredPartitionIterator makePartitionIterator(final ColumnFilter columnFilter, final DataRange dataRange) + { + AbstractBounds keyRange = dataRange.keyRange(); + + PartitionPosition left = keyRange.left; + PartitionPosition right = keyRange.right; + if (left.isMinimum()) + left = null; + if (right.isMinimum()) + right = null; + + boolean isBound = keyRange instanceof Bounds; + boolean includeStart = isBound || keyRange instanceof IncludingExcludingBounds; + boolean includeStop = isBound || keyRange instanceof Range; + + Trie subMap = mergedTrie.subtrie(left, includeStart, right, includeStop); + + return new MemtableUnfilteredPartitionIterator(metadata(), + allocator.ensureOnHeap(), + subMap, + columnFilter, + dataRange); + } + + public Partition getPartition(DecoratedKey key) + { + int shardIndex = getShardForKey(key); + BTreePartitionData data = shards[shardIndex].data.get(key); + if (data != null) + return createPartition(metadata(), allocator.ensureOnHeap(), key, data); + else + return null; + } + + private static MemtablePartition createPartition(TableMetadata metadata, EnsureOnHeap ensureOnHeap, DecoratedKey key, BTreePartitionData data) + { + return new MemtablePartition(metadata, ensureOnHeap, key, data); + } + + private static MemtablePartition getPartitionFromTrieEntry(TableMetadata metadata, EnsureOnHeap ensureOnHeap, Map.Entry en) + { + DecoratedKey key = BufferDecoratedKey.fromByteComparable(en.getKey(), + BYTE_COMPARABLE_VERSION, + metadata.partitioner); + return createPartition(metadata, ensureOnHeap, key, en.getValue()); + } + + + public FlushCollection getFlushSet(PartitionPosition from, PartitionPosition to) + { + Trie toFlush = mergedTrie.subtrie(from, true, to, false); + long keySize = 0; + int keyCount = 0; + + for (Iterator> it = toFlush.entryIterator(); it.hasNext(); ) + { + Map.Entry en = it.next(); + ByteComparable byteComparable = v -> en.getKey().asPeekableBytes(BYTE_COMPARABLE_VERSION); + byte[] keyBytes = DecoratedKey.keyFromByteComparable(byteComparable, BYTE_COMPARABLE_VERSION, metadata().partitioner); + keySize += keyBytes.length; + keyCount++; + } + long partitionKeySize = keySize; + int partitionCount = keyCount; + + return new AbstractFlushCollection() + { + public Memtable memtable() + { + return TrieMemtable.this; + } + + public PartitionPosition from() + { + return from; + } + + public PartitionPosition to() + { + return to; + } + + public long partitionCount() + { + return partitionCount; + } + + public Iterator iterator() + { + return Iterators.transform(toFlush.entryIterator(), + // During flushing we are certain the memtable will remain at least until + // the flush completes. No copying to heap is necessary. + entry -> getPartitionFromTrieEntry(metadata(), EnsureOnHeap.NOOP, entry)); + } + + public long partitionKeySize() + { + return partitionKeySize; + } + }; + } + + static class MemtableShard + { + // The following fields are volatile as we have to make sure that when we + // collect results from all sub-ranges, the thread accessing the value + // is guaranteed to see the changes to the values. + + // The smallest timestamp for all partitions stored in this shard + private volatile long minTimestamp = Long.MAX_VALUE; + + private volatile long liveDataSize = 0; + + private volatile long currentOperations = 0; + + private ReentrantLock writeLock = new ReentrantLock(); + + // Content map for the given shard. This is implemented as a memtable trie which uses the prefix-free + // byte-comparable ByteSource representations of the keys to address the partitions. + // + // This map is used in a single-producer, multi-consumer fashion: only one thread will insert items but + // several threads may read from it and iterate over it. Iterators are created when a the first item of + // a flow is requested for example, and then used asynchronously when sub-sequent items are requested. + // + // Therefore, iterators should not throw ConcurrentModificationExceptions if the underlying map is modified + // during iteration, they should provide a weakly consistent view of the map instead. + // + // Also, this data is backed by memtable memory, when accessing it callers must specify if it can be accessed + // unsafely, meaning that the memtable will not be discarded as long as the data is used, or whether the data + // should be copied on heap for off-heap allocators. + @VisibleForTesting + final MemtableTrie data; + + private final ColumnsCollector columnsCollector; + + private final StatsCollector statsCollector; + + private final MemtableAllocator allocator; + + private final TrieMemtableMetricsView metrics; + + MemtableShard(int shardId, TableMetadataRef metadata, TrieMemtableMetricsView metrics) + { + this(metadata, AbstractAllocatorMemtable.MEMORY_POOL.newAllocator(), metrics); + } + + @VisibleForTesting + MemtableShard(TableMetadataRef metadata, MemtableAllocator allocator, TrieMemtableMetricsView metrics) + { + this.data = new MemtableTrie<>(BUFFER_TYPE); + this.columnsCollector = new AbstractMemtable.ColumnsCollector(metadata.get().regularAndStaticColumns()); + this.statsCollector = new AbstractMemtable.StatsCollector(); + this.allocator = allocator; + this.metrics = metrics; + } + + public long put(DecoratedKey key, PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) + { + BTreePartitionUpdater updater = new BTreePartitionUpdater(allocator, opGroup, indexer); + boolean locked = writeLock.tryLock(); + if (locked) + { + metrics.uncontendedPuts.inc(); + } + else + { + metrics.contendedPuts.inc(); + long lockStartTime = System.nanoTime(); + writeLock.lock(); + metrics.contentionTime.addNano(System.nanoTime() - lockStartTime); + } + try + { + try + { + long onHeap = data.sizeOnHeap(); + long offHeap = data.sizeOffHeap(); + // Use the fast recursive put if we know the key is small enough to not cause a stack overflow. + try + { + data.putSingleton(key, + update, + updater::mergePartitions, + key.getKeyLength() < MAX_RECURSIVE_KEY_LENGTH); + } + catch (MemtableTrie.SpaceExhaustedException e) + { + // This should never really happen as a flush would be triggered long before this limit is reached. + throw Throwables.propagate(e); + } + allocator.offHeap().adjust(data.sizeOffHeap() - offHeap, opGroup); + allocator.onHeap().adjust(data.sizeOnHeap() - onHeap, opGroup); + } + finally + { + updateMinTimestamp(update.stats().minTimestamp); + updateLiveDataSize(updater.dataSize); + updateCurrentOperations(update.operationCount()); + + // TODO: lambov 2021-03-30: check if stats are further optimisable + columnsCollector.update(update.columns()); + statsCollector.update(update.stats()); + } + } + finally + { + writeLock.unlock(); + } + return updater.colUpdateTimeDelta; + } + + public boolean isEmpty() + { + return data.isEmpty(); + } + + private void updateMinTimestamp(long timestamp) + { + if (timestamp < minTimestamp) + minTimestamp = timestamp; + } + + void updateLiveDataSize(long size) + { + liveDataSize = liveDataSize + size; + } + + private void updateCurrentOperations(long op) + { + currentOperations = currentOperations + op; + } + + public int size() + { + return data.valuesCount(); + } + + long minTimestamp() + { + return minTimestamp; + } + + long liveDataSize() + { + return liveDataSize; + } + + long currentOperations() + { + return currentOperations; + } + } + + static class MemtableUnfilteredPartitionIterator extends AbstractUnfilteredPartitionIterator implements Memtable.MemtableUnfilteredPartitionIterator + { + private final TableMetadata metadata; + private final EnsureOnHeap ensureOnHeap; + private final Trie source; + private final Iterator> iter; + private final ColumnFilter columnFilter; + private final DataRange dataRange; + + public MemtableUnfilteredPartitionIterator(TableMetadata metadata, + EnsureOnHeap ensureOnHeap, + Trie source, + ColumnFilter columnFilter, + DataRange dataRange) + { + this.metadata = metadata; + this.ensureOnHeap = ensureOnHeap; + this.iter = source.entryIterator(); + this.source = source; + this.columnFilter = columnFilter; + this.dataRange = dataRange; + } + + public int getMinLocalDeletionTime() + { + int minLocalDeletionTime = Integer.MAX_VALUE; + for (BTreePartitionData partition : source.values()) + minLocalDeletionTime = Math.min(minLocalDeletionTime, partition.stats.minLocalDeletionTime); + + return minLocalDeletionTime; + } + + public TableMetadata metadata() + { + return metadata; + } + + public boolean hasNext() + { + return iter.hasNext(); + } + + public UnfilteredRowIterator next() + { + Partition partition = getPartitionFromTrieEntry(metadata(), ensureOnHeap, iter.next()); + DecoratedKey key = partition.partitionKey(); + ClusteringIndexFilter filter = dataRange.clusteringIndexFilter(key); + + return filter.getUnfilteredRowIterator(columnFilter, partition); + } + } + + static class MemtablePartition extends ImmutableBTreePartition + { + + private final EnsureOnHeap ensureOnHeap; + + private MemtablePartition(TableMetadata table, EnsureOnHeap ensureOnHeap, DecoratedKey key, BTreePartitionData data) + { + super(table, key, data); + this.ensureOnHeap = ensureOnHeap; + } + + @Override + protected boolean canHaveShadowedData() + { + // The BtreePartitionData we store in the memtable are build iteratively by BTreePartitionData.add(), which + // doesn't make sure there isn't shadowed data, so we'll need to eliminate any. + return true; + } + + + @Override + public DeletionInfo deletionInfo() + { + return ensureOnHeap.applyToDeletionInfo(super.deletionInfo()); + } + + @Override + public Row staticRow() + { + return ensureOnHeap.applyToStatic(super.staticRow()); + } + + @Override + public DecoratedKey partitionKey() + { + return ensureOnHeap.applyToPartitionKey(super.partitionKey()); + } + + @Override + public Row getRow(Clustering clustering) + { + return ensureOnHeap.applyToRow(super.getRow(clustering)); + } + + @Override + public Row lastRow() + { + return ensureOnHeap.applyToRow(super.lastRow()); + } + + @Override + public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices slices, boolean reversed) + { + return unfilteredIterator(holder(), selection, slices, reversed); + } + + @Override + public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, NavigableSet> clusteringsInQueryOrder, boolean reversed) + { + return ensureOnHeap + .applyToPartition(super.unfilteredIterator(selection, clusteringsInQueryOrder, reversed)); + } + + @Override + public UnfilteredRowIterator unfilteredIterator() + { + return unfilteredIterator(ColumnFilter.selection(super.columns()), Slices.ALL, false); + } + + @Override + public UnfilteredRowIterator unfilteredIterator(BTreePartitionData current, ColumnFilter selection, Slices slices, boolean reversed) + { + return ensureOnHeap + .applyToPartition(super.unfilteredIterator(current, selection, slices, reversed)); + } + + @Override + public Iterator iterator() + { + return ensureOnHeap.applyToPartition(super.iterator()); + } + } + + static class Factory implements Memtable.Factory + { + public Memtable create(AtomicReference commitLogLowerBound, + TableMetadataRef metadaRef, + Owner owner) + { + return new TrieMemtable(commitLogLowerBound, metadaRef, owner); + } + + @Override + public TableMetrics.ReleasableMetric createMemtableMetrics(TableMetadataRef metadataRef) + { + TrieMemtableMetricsView metrics = TrieMemtableMetricsView.getOrCreate(metadataRef.keyspace, metadataRef.name); + return metrics::release; + } + } + + @VisibleForTesting + public static class TrieMemtableConfig implements TrieMemtableConfigMXBean + { + @Override + public void setShardCount(String shardCount) + { + if ("auto".equalsIgnoreCase(shardCount)) + { + SHARD_COUNT = FBUtilities.getAvailableProcessors(); + } + else + { + try + { + SHARD_COUNT = Integer.valueOf(shardCount); + } + catch (NumberFormatException ex) + { + logger.warn("Unable to parse {} as valid value for shard count", shardCount); + return; + } + } + logger.info("Requested setting shard count to {}; set to: {}", shardCount, SHARD_COUNT); + } + + @Override + public String getShardCount() + { + return "" + SHARD_COUNT; + } + } +} diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtableConfigMXBean.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtableConfigMXBean.java new file mode 100644 index 000000000000..85123666ad91 --- /dev/null +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtableConfigMXBean.java @@ -0,0 +1,26 @@ +/* + * 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.memtable; + +public interface TrieMemtableConfigMXBean +{ + public void setShardCount(String numShards); + + public String getShardCount(); +} diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java index 1d6603eec2f2..7e3fdc5eec4f 100644 --- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java +++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java @@ -36,12 +36,9 @@ public abstract class AbstractBTreePartition implements Partition, Iterable { - protected static final Holder EMPTY = new Holder(RegularAndStaticColumns.NONE, BTree.empty(), DeletionInfo.LIVE, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS); - public static final long HOLDER_UNSHARED_HEAP_SIZE = ObjectSizes.measure(EMPTY); - protected final DecoratedKey partitionKey; - protected abstract Holder holder(); + protected abstract BTreePartitionData holder(); protected abstract boolean canHaveShadowedData(); protected AbstractBTreePartition(DecoratedKey partitionKey) @@ -49,25 +46,6 @@ protected AbstractBTreePartition(DecoratedKey partitionKey) this.partitionKey = partitionKey; } - protected static final class Holder - { - final RegularAndStaticColumns columns; - final DeletionInfo deletionInfo; - // the btree of rows - final Object[] tree; - final Row staticRow; - final EncodingStats stats; - - Holder(RegularAndStaticColumns columns, Object[] tree, DeletionInfo deletionInfo, Row staticRow, EncodingStats stats) - { - this.columns = columns; - this.tree = tree; - this.deletionInfo = deletionInfo; - this.staticRow = staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow; - this.stats = stats; - } - } - public DeletionInfo deletionInfo() { return holder().deletionInfo; @@ -80,13 +58,13 @@ public Row staticRow() public boolean isEmpty() { - Holder holder = holder(); + BTreePartitionData holder = holder(); return holder.deletionInfo.isLive() && BTree.isEmpty(holder.tree) && holder.staticRow.isEmpty(); } public boolean hasRows() { - Holder holder = holder(); + BTreePartitionData holder = holder(); return !BTree.isEmpty(holder.tree); } @@ -115,7 +93,7 @@ public EncodingStats stats() public Row getRow(Clustering clustering) { ColumnFilter columns = ColumnFilter.selection(columns()); - Holder holder = holder(); + BTreePartitionData holder = holder(); if (clustering == Clustering.STATIC_CLUSTERING) { @@ -145,7 +123,7 @@ public Row getRow(Clustering clustering) return row.filter(columns, activeDeletion, true, metadata()); } - private Row staticRow(Holder current, ColumnFilter columns, boolean setActiveDeletionToRow) + private Row staticRow(BTreePartitionData current, ColumnFilter columns, boolean setActiveDeletionToRow) { DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion(); if (columns.fetchedColumns().statics.isEmpty() || (current.staticRow.isEmpty() && partitionDeletion.isLive())) @@ -178,7 +156,7 @@ public UnfilteredRowIterator unfilteredIterator(ColumnFilter selection, Slices s return unfilteredIterator(holder(), selection, slices, reversed); } - public UnfilteredRowIterator unfilteredIterator(Holder current, ColumnFilter selection, Slices slices, boolean reversed) + public UnfilteredRowIterator unfilteredIterator(BTreePartitionData current, ColumnFilter selection, Slices slices, boolean reversed) { Row staticRow = staticRow(current, selection, false); if (slices.size() == 0) @@ -192,7 +170,7 @@ public UnfilteredRowIterator unfilteredIterator(Holder current, ColumnFilter sel : new SlicesIterator(selection, slices, reversed, current, staticRow); } - private UnfilteredRowIterator sliceIterator(ColumnFilter selection, Slice slice, boolean reversed, Holder current, Row staticRow) + private UnfilteredRowIterator sliceIterator(ColumnFilter selection, Slice slice, boolean reversed, BTreePartitionData current, Row staticRow) { ClusteringBound start = slice.start().isBottom() ? null : slice.start(); ClusteringBound end = slice.end().isTop() ? null : slice.end(); @@ -202,7 +180,7 @@ private UnfilteredRowIterator sliceIterator(ColumnFilter selection, Slice slice, } private RowAndDeletionMergeIterator merge(Iterator rowIter, Iterator deleteIter, - ColumnFilter selection, boolean reversed, Holder current, Row staticRow) + ColumnFilter selection, boolean reversed, BTreePartitionData current, Row staticRow) { return new RowAndDeletionMergeIterator(metadata(), partitionKey(), current.deletionInfo.getPartitionDeletion(), selection, staticRow, reversed, current.stats, @@ -212,10 +190,10 @@ private RowAndDeletionMergeIterator merge(Iterator rowIter, Iterator currentSlice; - private SlicesIterator(ColumnFilter selection, Slices slices, boolean isReversed, Holder current, Row staticRow) + private SlicesIterator(ColumnFilter selection, Slices slices, boolean isReversed, BTreePartitionData current, Row staticRow) { super(current, staticRow, selection, isReversed); this.slices = slices; @@ -276,7 +254,7 @@ private class ClusteringsIterator extends AbstractIterator private ClusteringsIterator(ColumnFilter selection, NavigableSet> clusteringsInQueryOrder, boolean isReversed, - Holder current, + BTreePartitionData current, Row staticRow) { super(current, staticRow, selection, isReversed); @@ -319,12 +297,12 @@ private Iterator nextIterator(Clustering next) } } - protected static Holder build(UnfilteredRowIterator iterator, int initialRowCapacity) + protected static BTreePartitionData build(UnfilteredRowIterator iterator, int initialRowCapacity) { return build(iterator, initialRowCapacity, true); } - protected static Holder build(UnfilteredRowIterator iterator, int initialRowCapacity, boolean ordered) + protected static BTreePartitionData build(UnfilteredRowIterator iterator, int initialRowCapacity, boolean ordered) { TableMetadata metadata = iterator.metadata(); RegularAndStaticColumns columns = iterator.columns(); @@ -346,12 +324,12 @@ protected static Holder build(UnfilteredRowIterator iterator, int initialRowCapa if (reversed) builder.reverse(); - return new Holder(columns, builder.build(), deletionBuilder.build(), iterator.staticRow(), iterator.stats()); + return new BTreePartitionData(columns, builder.build(), deletionBuilder.build(), iterator.staticRow(), iterator.stats()); } // Note that when building with a RowIterator, deletion will generally be LIVE, but we allow to pass it nonetheless because PartitionUpdate // passes a MutableDeletionInfo that it mutates later. - protected static Holder build(RowIterator rows, DeletionInfo deletion, boolean buildEncodingStats, int initialRowCapacity) + protected static BTreePartitionData build(RowIterator rows, DeletionInfo deletion, boolean buildEncodingStats, int initialRowCapacity) { TableMetadata metadata = rows.metadata(); RegularAndStaticColumns columns = rows.columns(); @@ -369,7 +347,7 @@ protected static Holder build(RowIterator rows, DeletionInfo deletion, boolean b Object[] tree = builder.build(); EncodingStats stats = buildEncodingStats ? EncodingStats.Collector.collect(staticRow, BTree.iterator(tree), deletion) : EncodingStats.NO_STATS; - return new Holder(columns, tree, deletion, staticRow, stats); + return new BTreePartitionData(columns, tree, deletion, staticRow, stats); } @Override diff --git a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java index 801d9e2338f7..d5c82845f15c 100644 --- a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java +++ b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java @@ -18,25 +18,20 @@ package org.apache.cassandra.db.partitions; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Iterator; -import java.util.List; import java.util.NavigableSet; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import org.apache.cassandra.index.transactions.UpdateTransaction; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.*; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.rows.*; -import org.apache.cassandra.index.transactions.UpdateTransaction; import org.apache.cassandra.utils.ObjectSizes; -import org.apache.cassandra.utils.btree.BTree; -import org.apache.cassandra.utils.btree.UpdateFunction; import org.apache.cassandra.utils.concurrent.OpOrder; -import org.apache.cassandra.utils.memory.HeapAllocator; import org.apache.cassandra.utils.memory.MemtableAllocator; /** @@ -67,7 +62,7 @@ public final class AtomicBTreePartition extends AbstractBTreePartition // CLOCK_GRANULARITY = 1^9ns >> CLOCK_SHIFT == 132us == (1/7.63)ms private static final AtomicIntegerFieldUpdater wasteTrackerUpdater = AtomicIntegerFieldUpdater.newUpdater(AtomicBTreePartition.class, "wasteTracker"); - private static final AtomicReferenceFieldUpdater refUpdater = AtomicReferenceFieldUpdater.newUpdater(AtomicBTreePartition.class, Holder.class, "ref"); + private static final AtomicReferenceFieldUpdater refUpdater = AtomicReferenceFieldUpdater.newUpdater(AtomicBTreePartition.class, BTreePartitionData.class, "ref"); /** * (clock + allocation) granularity are combined to give us an acceptable (waste) allocation rate that is defined by @@ -80,7 +75,7 @@ public final class AtomicBTreePartition extends AbstractBTreePartition private volatile int wasteTracker = TRACKER_NEVER_WASTED; private final MemtableAllocator allocator; - private volatile Holder ref; + private volatile BTreePartitionData ref; private final TableMetadataRef metadata; @@ -90,10 +85,10 @@ public AtomicBTreePartition(TableMetadataRef metadata, DecoratedKey partitionKey super(partitionKey); this.metadata = metadata; this.allocator = allocator; - this.ref = EMPTY; + this.ref = BTreePartitionData.EMPTY; } - protected Holder holder() + protected BTreePartitionData holder() { return ref; } @@ -108,90 +103,71 @@ protected boolean canHaveShadowedData() return true; } - private long[] addAllWithSizeDeltaInternal(RowUpdater updater, PartitionUpdate update, UpdateTransaction indexer) - { - Holder current = ref; - updater.ref = current; - updater.reset(); - - if (!update.deletionInfo().getPartitionDeletion().isLive()) - indexer.onPartitionDeletion(update.deletionInfo().getPartitionDeletion()); - - if (update.deletionInfo().hasRanges()) - update.deletionInfo().rangeIterator(false).forEachRemaining(indexer::onRangeTombstone); - - DeletionInfo deletionInfo; - if (update.deletionInfo().mayModify(current.deletionInfo)) - { - if (updater.inputDeletionInfoCopy == null) - updater.inputDeletionInfoCopy = update.deletionInfo().copy(HeapAllocator.instance); - - deletionInfo = current.deletionInfo.mutableCopy().add(updater.inputDeletionInfoCopy); - updater.allocated(deletionInfo.unsharedHeapSize() - current.deletionInfo.unsharedHeapSize()); - } - else - { - deletionInfo = current.deletionInfo; - } - - RegularAndStaticColumns columns = update.columns().mergeTo(current.columns); - updater.allocated(columns.unsharedHeapSize() - current.columns.unsharedHeapSize()); - Row newStatic = update.staticRow(); - Row staticRow = newStatic.isEmpty() - ? current.staticRow - : (current.staticRow.isEmpty() ? updater.apply(newStatic) : updater.apply(current.staticRow, newStatic)); - Object[] tree = BTree.update(current.tree, update.metadata().comparator, update, update.rowCount(), updater); - EncodingStats newStats = current.stats.mergeWith(update.stats()); - updater.allocated(newStats.unsharedHeapSize() - current.stats.unsharedHeapSize()); - - if (tree != null && refUpdater.compareAndSet(this, current, new Holder(columns, tree, deletionInfo, staticRow, newStats))) - { - updater.finish(); - return new long[]{ updater.dataSize, updater.colUpdateTimeDelta }; - } - else - { - return null; - } - } /** * Adds a given update to this in-memtable partition. * * @return an array containing first the difference in size seen after merging the updates, and second the minimum * time detla between updates. */ - public long[] addAllWithSizeDelta(final PartitionUpdate update, OpOrder.Group writeOp, UpdateTransaction indexer) + public BTreePartitionUpdater addAll(final PartitionUpdate update, OpOrder.Group writeOp, UpdateTransaction indexer) { - RowUpdater updater = new RowUpdater(this, allocator, writeOp, indexer); - try + return new Updater(allocator, writeOp, indexer).addAll(update); + } + + class Updater extends BTreePartitionUpdater + { + BTreePartitionData current; + + public Updater(MemtableAllocator allocator, OpOrder.Group writeOp, UpdateTransaction indexer) { - boolean shouldLock = shouldLock(writeOp); - indexer.start(); + super(allocator, writeOp, indexer); + } - while (true) + Updater addAll(final PartitionUpdate update) + { + try { - if (shouldLock) + boolean shouldLock = shouldLock(writeOp); + indexer.start(); + + while (true) { - synchronized (this) + if (shouldLock) { - long[] result = addAllWithSizeDeltaInternal(updater, update, indexer); - if (result != null) - return result; + synchronized (this) + { + if (tryUpdateData(update)) + return this; + } } - } - else - { - long[] result = addAllWithSizeDeltaInternal(updater, update, indexer); - if (result != null) - return result; + else + { + if (tryUpdateData(update)) + return this; - shouldLock = shouldLock(updater.heapSize, writeOp); + shouldLock = shouldLock(heapSize, writeOp); + } } } + finally + { + indexer.commit(); + reportAllocatedMemory(); + } } - finally + + private boolean tryUpdateData(PartitionUpdate update) { - indexer.commit(); + current = ref; + this.dataSize = 0; + this.heapSize = 0; + BTreePartitionData result = makeMergedPartition(current, update); + return refUpdater.compareAndSet(AtomicBTreePartition.this, current, result); + } + + public boolean abortEarly() + { + return ref != current; } } @@ -244,7 +220,7 @@ public UnfilteredRowIterator unfilteredIterator() } @Override - public UnfilteredRowIterator unfilteredIterator(Holder current, ColumnFilter selection, Slices slices, boolean reversed) + public UnfilteredRowIterator unfilteredIterator(BTreePartitionData current, ColumnFilter selection, Slices slices, boolean reversed) { return allocator.ensureOnHeap().applyToPartition(super.unfilteredIterator(current, selection, slices, reversed)); } @@ -328,94 +304,4 @@ private static int avoidReservedValues(int wasteTracker) return wasteTracker + 1; return wasteTracker; } - - // the function we provide to the btree utilities to perform any column replacements - private static final class RowUpdater implements UpdateFunction - { - final AtomicBTreePartition updating; - final MemtableAllocator allocator; - final OpOrder.Group writeOp; - final UpdateTransaction indexer; - Holder ref; - Row.Builder regularBuilder; - long dataSize; - long heapSize; - long colUpdateTimeDelta = Long.MAX_VALUE; - List inserted; // TODO: replace with walk of aborted BTree - - DeletionInfo inputDeletionInfoCopy = null; - - private RowUpdater(AtomicBTreePartition updating, MemtableAllocator allocator, OpOrder.Group writeOp, UpdateTransaction indexer) - { - this.updating = updating; - this.allocator = allocator; - this.writeOp = writeOp; - this.indexer = indexer; - } - - private Row.Builder builder(Clustering clustering) - { - boolean isStatic = clustering == Clustering.STATIC_CLUSTERING; - // We know we only insert/update one static per PartitionUpdate, so no point in saving the builder - if (isStatic) - return allocator.rowBuilder(writeOp); - - if (regularBuilder == null) - regularBuilder = allocator.rowBuilder(writeOp); - return regularBuilder; - } - - public Row apply(Row insert) - { - Row data = Rows.copy(insert, builder(insert.clustering())).build(); - indexer.onInserted(insert); - - this.dataSize += data.dataSize(); - allocated(data.unsharedHeapSizeExcludingData()); - if (inserted == null) - inserted = new ArrayList<>(); - inserted.add(data); - return data; - } - - public Row apply(Row existing, Row update) - { - Row.Builder builder = builder(existing.clustering()); - colUpdateTimeDelta = Math.min(colUpdateTimeDelta, Rows.merge(existing, update, builder)); - - Row reconciled = builder.build(); - - indexer.onUpdated(existing, reconciled); - - dataSize += reconciled.dataSize() - existing.dataSize(); - allocated(reconciled.unsharedHeapSizeExcludingData() - existing.unsharedHeapSizeExcludingData()); - if (inserted == null) - inserted = new ArrayList<>(); - inserted.add(reconciled); - - return reconciled; - } - - protected void reset() - { - this.dataSize = 0; - this.heapSize = 0; - if (inserted != null) - inserted.clear(); - } - public boolean abortEarly() - { - return updating.ref != ref; - } - - public void allocated(long heapSize) - { - this.heapSize += heapSize; - } - - protected void finish() - { - allocator.onHeap().adjust(heapSize, writeOp); - } - } } diff --git a/src/java/org/apache/cassandra/db/partitions/BTreePartitionData.java b/src/java/org/apache/cassandra/db/partitions/BTreePartitionData.java new file mode 100644 index 000000000000..1f0320d37fdc --- /dev/null +++ b/src/java/org/apache/cassandra/db/partitions/BTreePartitionData.java @@ -0,0 +1,58 @@ +/* + * 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.partitions; + +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.btree.BTree; + +/** + * Holder of the content of a partition, see AbstractBTreePartition. + * When updating a partition one holder is swapped for another atomically. + */ +public final class BTreePartitionData +{ + public static final BTreePartitionData EMPTY = new BTreePartitionData(RegularAndStaticColumns.NONE, BTree.empty(), DeletionInfo.LIVE, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS); + public static final long UNSHARED_HEAP_SIZE = ObjectSizes.measure(EMPTY); + + + final RegularAndStaticColumns columns; + final DeletionInfo deletionInfo; + // the btree of rows + final Object[] tree; + final Row staticRow; + public final EncodingStats stats; + + BTreePartitionData(RegularAndStaticColumns columns, + Object[] tree, + DeletionInfo deletionInfo, + Row staticRow, + EncodingStats stats) + { + this.columns = columns; + this.tree = tree; + this.deletionInfo = deletionInfo; + this.staticRow = staticRow == null ? Rows.EMPTY_STATIC_ROW : staticRow; + this.stats = stats; + } +} diff --git a/src/java/org/apache/cassandra/db/partitions/BTreePartitionUpdater.java b/src/java/org/apache/cassandra/db/partitions/BTreePartitionUpdater.java new file mode 100644 index 000000000000..75253aa24019 --- /dev/null +++ b/src/java/org/apache/cassandra/db/partitions/BTreePartitionUpdater.java @@ -0,0 +1,168 @@ +/* + * 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.partitions; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.utils.btree.BTree; +import org.apache.cassandra.utils.btree.UpdateFunction; +import org.apache.cassandra.utils.concurrent.OpOrder; +import org.apache.cassandra.utils.memory.HeapAllocator; +import org.apache.cassandra.utils.memory.MemtableAllocator; + +/** + * the function we provide to the trie and btree utilities to perform any row and column replacements + */ +public class BTreePartitionUpdater implements UpdateFunction +{ + final MemtableAllocator allocator; + final OpOrder.Group writeOp; + final UpdateTransaction indexer; + Row.Builder regularBuilder; + public long dataSize; + long heapSize; + public long colUpdateTimeDelta = Long.MAX_VALUE; + + public BTreePartitionUpdater(MemtableAllocator allocator, OpOrder.Group writeOp, UpdateTransaction indexer) + { + this.allocator = allocator; + this.writeOp = writeOp; + this.indexer = indexer; + this.heapSize = 0; + this.dataSize = 0; + } + + private Row.Builder builder(Clustering clustering) + { + boolean isStatic = clustering == Clustering.STATIC_CLUSTERING; + // We know we only insert/update one static per PartitionUpdate, so no point in saving the builder + if (isStatic) + return allocator.rowBuilder(writeOp); + + if (regularBuilder == null) + regularBuilder = allocator.rowBuilder(writeOp); + return regularBuilder; + } + + public Row apply(Row insert) + { + Row data = Rows.copy(insert, builder(insert.clustering())).build(); + indexer.onInserted(insert); + + this.dataSize += data.dataSize(); + allocated(data.unsharedHeapSizeExcludingData()); + return data; + } + + public Row apply(Row existing, Row update) + { + Row.Builder builder = builder(existing.clustering()); + colUpdateTimeDelta = Math.min(colUpdateTimeDelta, Rows.merge(existing, update, builder)); + + Row reconciled = builder.build(); + + indexer.onUpdated(existing, reconciled); + + dataSize += reconciled.dataSize() - existing.dataSize(); + allocated(reconciled.unsharedHeapSizeExcludingData() - existing.unsharedHeapSizeExcludingData()); + + return reconciled; + } + + private DeletionInfo apply(DeletionInfo existing, DeletionInfo update) + { + if (update.isLive() || !update.mayModify(existing)) + return existing; + + if (!update.getPartitionDeletion().isLive()) + indexer.onPartitionDeletion(update.getPartitionDeletion()); + + if (update.hasRanges()) + update.rangeIterator(false).forEachRemaining(indexer::onRangeTombstone); + + // Like for rows, we have to clone the update in case internal buffers (when it has range tombstones) reference + // memory we shouldn't hold into. But we don't ever store this off-heap currently so we just default to the + // HeapAllocator (rather than using 'allocator'). + DeletionInfo newInfo = existing.mutableCopy().add(update.copy(HeapAllocator.instance)); + allocated(newInfo.unsharedHeapSize() - existing.unsharedHeapSize()); + return newInfo; + } + + public BTreePartitionData mergePartitions(BTreePartitionData current, final PartitionUpdate update) + { + if (current == null) + { + current = BTreePartitionData.EMPTY; + this.allocated(BTreePartitionData.UNSHARED_HEAP_SIZE); + } + + try + { + indexer.start(); + + return makeMergedPartition(current, update); + } + finally + { + indexer.commit(); + reportAllocatedMemory(); + } + } + + protected BTreePartitionData makeMergedPartition(BTreePartitionData current, PartitionUpdate update) + { + DeletionInfo newDeletionInfo = apply(current.deletionInfo, update.deletionInfo()); + + RegularAndStaticColumns columns = current.columns; + RegularAndStaticColumns newColumns = update.columns().mergeTo(columns); + allocated(newColumns.unsharedHeapSize() - columns.unsharedHeapSize()); + Row newStatic = update.staticRow(); + newStatic = newStatic.isEmpty() + ? current.staticRow + : (current.staticRow.isEmpty() + ? this.apply(newStatic) + : this.apply(current.staticRow, newStatic)); + + Object[] tree = BTree.update(current.tree, update.metadata().comparator, update, update.rowCount(), this); + EncodingStats newStats = current.stats.mergeWith(update.stats()); + allocated(newStats.unsharedHeapSize() - current.stats.unsharedHeapSize()); + + return new BTreePartitionData(newColumns, tree, newDeletionInfo, newStatic, newStats); + } + + public boolean abortEarly() + { + return false; + } + + public void allocated(long heapSize) + { + this.heapSize += heapSize; + } + + public void reportAllocatedMemory() + { + allocator.onHeap().adjust(heapSize, writeOp); + } +} diff --git a/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java index 2183a9852a49..f09f75aa5864 100644 --- a/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java +++ b/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java @@ -40,7 +40,7 @@ public class CachedBTreePartition extends ImmutableBTreePartition implements Cac private CachedBTreePartition(TableMetadata metadata, DecoratedKey partitionKey, - Holder holder, + BTreePartitionData holder, int createdAtInSec, int cachedLiveRows, int rowsWithNonExpiringCells) @@ -80,7 +80,7 @@ public static CachedBTreePartition create(UnfilteredRowIterator iterator, int no */ public static CachedBTreePartition create(UnfilteredRowIterator iterator, int initialRowCapacity, int nowInSec) { - Holder holder = ImmutableBTreePartition.build(iterator, initialRowCapacity); + BTreePartitionData holder = ImmutableBTreePartition.build(iterator, initialRowCapacity); int cachedLiveRows = 0; int rowsWithNonExpiringCells = 0; @@ -180,7 +180,7 @@ public CachedPartition deserialize(DataInputPlus in) throws IOException UnfilteredRowIteratorSerializer.Header header = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(metadata, null, in, version, DeserializationHelper.Flag.LOCAL); assert !header.isReversed && header.rowEstimate >= 0; - Holder holder; + BTreePartitionData holder; try (UnfilteredRowIterator partition = UnfilteredRowIteratorSerializer.serializer.deserialize(in, version, metadata, DeserializationHelper.Flag.LOCAL, header)) { holder = ImmutableBTreePartition.build(partition, header.rowEstimate); diff --git a/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java index 5139d40134b9..661725566002 100644 --- a/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java +++ b/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java @@ -27,7 +27,7 @@ public class ImmutableBTreePartition extends AbstractBTreePartition { - protected final Holder holder; + protected final BTreePartitionData holder; protected final TableMetadata metadata; public ImmutableBTreePartition(TableMetadata metadata, @@ -40,12 +40,12 @@ public ImmutableBTreePartition(TableMetadata metadata, { super(partitionKey); this.metadata = metadata; - this.holder = new Holder(columns, tree, deletionInfo, staticRow, stats); + this.holder = new BTreePartitionData(columns, tree, deletionInfo, staticRow, stats); } protected ImmutableBTreePartition(TableMetadata metadata, DecoratedKey partitionKey, - Holder holder) + BTreePartitionData holder) { super(partitionKey); this.metadata = metadata; @@ -119,7 +119,7 @@ public TableMetadata metadata() return metadata; } - protected Holder holder() + protected BTreePartitionData holder() { return holder; } diff --git a/src/java/org/apache/cassandra/db/partitions/Partition.java b/src/java/org/apache/cassandra/db/partitions/Partition.java index b6297a1c3979..4fb5148cb129 100644 --- a/src/java/org/apache/cassandra/db/partitions/Partition.java +++ b/src/java/org/apache/cassandra/db/partitions/Partition.java @@ -50,6 +50,11 @@ public interface Partition */ public boolean isEmpty(); + /** + * Whether the partition object has rows. This may be true but partition still be non-empty if it has a deletion. + */ + boolean hasRows(); + /** * Returns the row corresponding to the provided clustering, or null if there is not such row. * diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java b/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java index 6d376401464c..a9beb2a11d0f 100644 --- a/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java +++ b/src/java/org/apache/cassandra/db/partitions/PartitionStatisticsCollector.java @@ -23,6 +23,7 @@ public interface PartitionStatisticsCollector { public void update(LivenessInfo info); + public void updatePartitionDeletion(DeletionTime dt); public void update(DeletionTime deletionTime); public void update(Cell cell); public void updateColumnSetPerRow(long columnSetInRow); diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java index ce1a8508c203..8dffa5e17a27 100644 --- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java +++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java @@ -61,7 +61,7 @@ public class PartitionUpdate extends AbstractBTreePartition public static final PartitionUpdateSerializer serializer = new PartitionUpdateSerializer(); - private final Holder holder; + private final BTreePartitionData holder; private final DeletionInfo deletionInfo; private final TableMetadata metadata; @@ -69,7 +69,7 @@ public class PartitionUpdate extends AbstractBTreePartition private PartitionUpdate(TableMetadata metadata, DecoratedKey key, - Holder holder, + BTreePartitionData holder, MutableDeletionInfo deletionInfo, boolean canHaveShadowedData) { @@ -91,7 +91,7 @@ private PartitionUpdate(TableMetadata metadata, public static PartitionUpdate emptyUpdate(TableMetadata metadata, DecoratedKey key) { MutableDeletionInfo deletionInfo = MutableDeletionInfo.live(); - Holder holder = new Holder(RegularAndStaticColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS); + BTreePartitionData holder = new BTreePartitionData(RegularAndStaticColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS); return new PartitionUpdate(metadata, key, holder, deletionInfo, false); } @@ -108,7 +108,7 @@ public static PartitionUpdate emptyUpdate(TableMetadata metadata, DecoratedKey k public static PartitionUpdate fullPartitionDelete(TableMetadata metadata, DecoratedKey key, long timestamp, int nowInSec) { MutableDeletionInfo deletionInfo = new MutableDeletionInfo(timestamp, nowInSec); - Holder holder = new Holder(RegularAndStaticColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS); + BTreePartitionData holder = new BTreePartitionData(RegularAndStaticColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS); return new PartitionUpdate(metadata, key, holder, deletionInfo, false); } @@ -125,7 +125,7 @@ public static PartitionUpdate fullPartitionDelete(TableMetadata metadata, Decora public static PartitionUpdate singleRowUpdate(TableMetadata metadata, DecoratedKey key, Row row, Row staticRow) { MutableDeletionInfo deletionInfo = MutableDeletionInfo.live(); - Holder holder = new Holder( + BTreePartitionData holder = new BTreePartitionData( new RegularAndStaticColumns( staticRow == null ? Columns.NONE : Columns.from(staticRow.columns()), row == null ? Columns.NONE : Columns.from(row.columns()) @@ -181,7 +181,7 @@ public static PartitionUpdate singleRowUpdate(TableMetadata metadata, ByteBuffer public static PartitionUpdate fromIterator(UnfilteredRowIterator iterator, ColumnFilter filter) { iterator = UnfilteredRowIterators.withOnlyQueriedData(iterator, filter); - Holder holder = build(iterator, 16); + BTreePartitionData holder = build(iterator, 16); MutableDeletionInfo deletionInfo = (MutableDeletionInfo) holder.deletionInfo; return new PartitionUpdate(iterator.metadata(), iterator.partitionKey(), holder, deletionInfo, false); } @@ -202,7 +202,7 @@ public static PartitionUpdate fromIterator(RowIterator iterator, ColumnFilter fi { iterator = RowIterators.withOnlyQueriedData(iterator, filter); MutableDeletionInfo deletionInfo = MutableDeletionInfo.live(); - Holder holder = build(iterator, deletionInfo, true, 16); + BTreePartitionData holder = build(iterator, deletionInfo, true, 16); return new PartitionUpdate(iterator.metadata(), iterator.partitionKey(), holder, deletionInfo, false); } @@ -357,7 +357,7 @@ public RegularAndStaticColumns columns() return holder.columns; } - protected Holder holder() + protected BTreePartitionData holder() { return holder; } @@ -670,7 +670,7 @@ public PartitionUpdate deserialize(DataInputPlus in, int version, Deserializatio MutableDeletionInfo deletionInfo = deletionBuilder.build(); return new PartitionUpdate(metadata, header.key, - new Holder(header.sHeader.columns(), rows.build(), deletionInfo, header.staticRow, header.sHeader.stats()), + new BTreePartitionData(header.sHeader.columns(), rows.build(), deletionInfo, header.staticRow, header.sHeader.stats()), deletionInfo, false); } @@ -765,7 +765,7 @@ private Builder(TableMetadata metadata, RegularAndStaticColumns columns, int initialRowCapacity, boolean canHaveShadowedData, - Holder holder) + BTreePartitionData holder) { this(metadata, key, columns, initialRowCapacity, canHaveShadowedData, holder.staticRow, holder.deletionInfo, holder.tree); } @@ -874,11 +874,11 @@ public PartitionUpdate build() isBuilt = true; return new PartitionUpdate(metadata, partitionKey(), - new Holder(columns, - merged, - deletionInfo, - staticRow, - newStats), + new BTreePartitionData(columns, + merged, + deletionInfo, + staticRow, + newStats), deletionInfo, canHaveShadowedData); } diff --git a/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java b/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java index d9e90367d2d3..09f3ae3bbf18 100644 --- a/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java +++ b/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java @@ -18,7 +18,6 @@ package org.apache.cassandra.db.partitions; import java.util.function.LongPredicate; -import java.util.function.Predicate; import org.apache.cassandra.db.*; import org.apache.cassandra.db.rows.*; diff --git a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java index a051ee1a9d26..e72463ffc608 100644 --- a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java +++ b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java @@ -149,7 +149,7 @@ protected UnfilteredRowIterator getReduced() return UnfilteredRowIterators.merge(toMerge, rowListener); } - protected void onKeyChange() + public void onKeyChange() { toMerge.clear(); for (int i = 0; i < iterators.size(); i++) @@ -215,7 +215,7 @@ protected UnfilteredRowIterator initializeIterator() }; } - protected void onKeyChange() + public void onKeyChange() { toMerge.clear(); } diff --git a/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java b/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java index 983e30f20728..dc975417cf72 100644 --- a/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java +++ b/src/java/org/apache/cassandra/db/repair/CassandraTableRepairManager.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.Future; @@ -28,13 +29,17 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.compaction.RepairFinishedCompactionTask; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; 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.repair.TableRepairManager; import org.apache.cassandra.repair.ValidationPartitionIterator; -import org.apache.cassandra.repair.Validator; +import org.apache.cassandra.repair.consistent.LocalSessions; +import org.apache.cassandra.service.ActiveRepairService; public class CassandraTableRepairManager implements TableRepairManager { @@ -58,9 +63,37 @@ public Future submitValidation(Callable validation) } @Override - public void incrementalSessionCompleted(UUID sessionID) + public synchronized void incrementalSessionCompleted(UUID sessionID) { - CompactionManager.instance.submitBackground(cfs); + LocalSessions sessions = ActiveRepairService.instance.consistent.local; + if (sessions.isSessionInProgress(sessionID)) + return; + + Set pendingRepairSSTables = cfs.getPendingRepairSSTables(sessionID); + if (pendingRepairSSTables.isEmpty()) + return; + + LifecycleTransaction txn = cfs.getTracker().tryModify(pendingRepairSSTables, OperationType.COMPACTION); + if (txn == null) + return; + + boolean isTransient = false; + for (SSTableReader sstable : pendingRepairSSTables) + { + if (sstable.isTransient()) + { + isTransient = true; + break; + } + } + + long repairedAt = sessions.getFinalSessionRepairedAt(sessionID); + RepairFinishedCompactionTask task = new RepairFinishedCompactionTask(cfs, + txn, + sessionID, + repairedAt, + isTransient); + task.run(); } @Override diff --git a/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java b/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java index 6f2256f370b8..81067ef7bbcc 100644 --- a/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java +++ b/src/java/org/apache/cassandra/db/repair/CassandraValidationIterator.java @@ -30,6 +30,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.Collections2; import com.google.common.collect.Maps; import org.slf4j.Logger; @@ -37,11 +38,8 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.compaction.AbstractCompactionStrategy; -import org.apache.cassandra.db.compaction.ActiveCompactionsTracker; import org.apache.cassandra.db.compaction.CompactionController; import org.apache.cassandra.db.compaction.CompactionIterator; -import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.db.lifecycle.View; @@ -50,11 +48,11 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.ScannerList; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.repair.ValidationPartitionIterator; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ActiveRepairService; -import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.UUIDGen; import org.apache.cassandra.utils.concurrent.Refs; @@ -102,9 +100,9 @@ public static int getDefaultGcBefore(ColumnFamilyStore cfs, int nowInSec) private static class ValidationCompactionIterator extends CompactionIterator { - public ValidationCompactionIterator(List scanners, ValidationCompactionController controller, int nowInSec, ActiveCompactionsTracker activeCompactions) + public ValidationCompactionIterator(List scanners, ValidationCompactionController controller, int nowInSec) { - super(OperationType.VALIDATION, scanners, controller, nowInSec, UUIDGen.getTimeUUID(), activeCompactions); + super(OperationType.VALIDATION, scanners, controller, nowInSec, UUIDGen.getTimeUUID()); } } @@ -166,7 +164,7 @@ else if (isIncremental) private final boolean isGlobalSnapshotValidation; private final boolean isSnapshotValidation; - private final AbstractCompactionStrategy.ScannerList scanners; + private final ScannerList scanners; private final ValidationCompactionController controller; private final CompactionIterator ci; @@ -198,12 +196,19 @@ public CassandraValidationIterator(ColumnFamilyStore cfs, Collection Collections2.transform(Range.normalize(ranges), Range::makeRowRange), + sstables); + Preconditions.checkArgument(sstables != null); + ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(parentId); if (prs != null) { @@ -216,8 +221,8 @@ public CassandraValidationIterator(ColumnFamilyStore cfs, Collection cis = CompactionManager.instance.active.getCompactionsForSSTable(sstable, OperationType.ANTICOMPACTION); - if (cis != null && !cis.isEmpty()) + Collection ops = CompactionManager.instance.active.getOperationsForSSTable(sstable, OperationType.ANTICOMPACTION); + if (ops != null && !ops.isEmpty()) { // todo: start tracking the parent repair session id that created the anticompaction to be able to give a better error messsage here: StringBuilder sb = new StringBuilder(); @@ -155,8 +155,10 @@ public boolean apply(SSTableReader sstable) sb.append(" has failed because it encountered intersecting sstables belonging to another incremental repair session. "); sb.append("This is caused by starting multiple conflicting incremental repairs at the same time. "); sb.append("Conflicting anticompactions: "); - for (CompactionInfo ci : cis) - sb.append(ci.getTaskId() == null ? "no compaction id" : ci.getTaskId()).append(':').append(ci.getSSTables()).append(','); + for (AbstractTableOperation.OperationProgress op : ops) + { + sb.append(op.operationId() == null ? "no compaction id" : op.operationId()).append(':').append(op.sstables()).append(','); + } throw new SSTableAcquisitionException(sb.toString()); } return true; @@ -364,7 +366,7 @@ public ListenableFuture run() List> tasks = new ArrayList<>(tables.size()); for (ColumnFamilyStore cfs : tables) { - cfs.forceBlockingFlush(); + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.REPAIR); ListenableFutureTask task = ListenableFutureTask.create(getAcquisitionCallable(cfs, tokenRanges.ranges(), prsId, acquireRetrySeconds, acquireSleepMillis)); executor.submit(task); tasks.add(task); diff --git a/src/java/org/apache/cassandra/db/rows/ArtificialBoundMarker.java b/src/java/org/apache/cassandra/db/rows/ArtificialBoundMarker.java new file mode 100644 index 000000000000..40402fece5fd --- /dev/null +++ b/src/java/org/apache/cassandra/db/rows/ArtificialBoundMarker.java @@ -0,0 +1,59 @@ +/* + * 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.rows; + +import java.util.Objects; + +import org.apache.cassandra.db.ClusteringBound; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.schema.TableMetadata; + +public class ArtificialBoundMarker extends RangeTombstoneBoundMarker +{ + public ArtificialBoundMarker(ClusteringBound bound) + { + super(bound, DeletionTime.LIVE); + assert bound.isArtificial(); + } + + @Override + public boolean equals(Object other) + { + if (this == other) + return true; + + if (!(other instanceof ArtificialBoundMarker)) + return false; + + ArtificialBoundMarker that = (ArtificialBoundMarker) other; + return Objects.equals(bound, that.bound); + } + + @Override + public int hashCode() + { + return Objects.hash(bound); + } + + @Override + public String toString(TableMetadata metadata) + { + return String.format("LowerBoundMarker %s", bound.toString(metadata)); + } +} diff --git a/src/java/org/apache/cassandra/db/rows/EncodingStats.java b/src/java/org/apache/cassandra/db/rows/EncodingStats.java index 37dd34e92185..dfaf672117ac 100644 --- a/src/java/org/apache/cassandra/db/rows/EncodingStats.java +++ b/src/java/org/apache/cassandra/db/rows/EncodingStats.java @@ -215,6 +215,12 @@ public void update(DeletionTime deletionTime) updateLocalDeletionTime(deletionTime.localDeletionTime()); } + @Override + public void updatePartitionDeletion(DeletionTime dt) + { + update(dt); + } + public void updateTimestamp(long timestamp) { isTimestampSet = true; diff --git a/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java index d8bd36f7e16c..8851b404faad 100644 --- a/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java +++ b/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java @@ -35,6 +35,8 @@ public abstract class LazilyInitializedUnfilteredRowIterator extends AbstractIte private UnfilteredRowIterator iterator; + private boolean closed = false; + public LazilyInitializedUnfilteredRowIterator(DecoratedKey partitionKey) { this.partitionKey = partitionKey; @@ -104,5 +106,11 @@ public void close() { if (iterator != null) iterator.close(); + closed = true; + } + + public boolean isClosed() + { + return closed; } } diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java index 5c28cd1b42f3..4b4d88db2663 100644 --- a/src/java/org/apache/cassandra/db/rows/Row.java +++ b/src/java/org/apache/cassandra/db/rows/Row.java @@ -839,7 +839,7 @@ protected ColumnData getReduced() } } - protected void onKeyChange() + public void onKeyChange() { column = null; versions.clear(); @@ -868,7 +868,7 @@ protected Cell getReduced() return merged; } - protected void onKeyChange() + public void onKeyChange() { merged = null; } diff --git a/src/java/org/apache/cassandra/db/rows/Rows.java b/src/java/org/apache/cassandra/db/rows/Rows.java index 82abb03d7e99..ce4d6dd48450 100644 --- a/src/java/org/apache/cassandra/db/rows/Rows.java +++ b/src/java/org/apache/cassandra/db/rows/Rows.java @@ -119,9 +119,8 @@ private static int unpackColumnCount(long v) * * @param row the row for which to collect stats. * @param collector the stats collector. - * @return the total number of cells in {@code row}. */ - public static int collectStats(Row row, PartitionStatisticsCollector collector) + public static void collectStats(Row row, PartitionStatisticsCollector collector) { assert !row.isEmpty(); @@ -131,7 +130,6 @@ public static int collectStats(Row row, PartitionStatisticsCollector collector) long result = row.accumulate(StatsAccumulation::accumulateOnColumnData, collector, 0); collector.updateColumnSetPerRow(StatsAccumulation.unpackColumnCount(result)); - return StatsAccumulation.unpackCellCount(result); } /** @@ -242,7 +240,7 @@ else if (cmp < 0) return null; } - protected void onKeyChange() + public void onKeyChange() { mergedData = null; Arrays.fill(inputDatas, null); @@ -273,7 +271,7 @@ public static Row merge(Row row1, Row row2) * * @return the smallest timestamp delta between corresponding rows from existing and update. A * timestamp delta being computed as the difference between the cells and DeletionTimes from {@code existing} - * and those in {@code existing}. + * and those in {@code update}. */ public static long merge(Row existing, Row update, diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java index 938a3eed114e..9b93c89f8454 100644 --- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java +++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java @@ -26,6 +26,7 @@ import org.apache.cassandra.db.*; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.io.sstable.format.big.ColumnIndex; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.TableMetadata; diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java index b6f425458dec..c90f47043fbd 100644 --- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java +++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java @@ -21,20 +21,19 @@ package org.apache.cassandra.db.rows; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Comparator; -import java.util.List; +import javax.annotation.Nonnull; -import org.apache.cassandra.db.marshal.ByteBufferAccessor; -import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.*; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.transform.RTBoundValidator; -import org.apache.cassandra.io.sstable.IndexInfo; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.SSTableReadsListener; +import org.apache.cassandra.io.sstable.format.big.BigTableRowIndexEntry; +import org.apache.cassandra.io.sstable.format.big.IndexInfo; import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.IteratorWithLowerBound; /** @@ -48,10 +47,11 @@ public class UnfilteredRowIteratorWithLowerBound extends LazilyInitializedUnfilteredRowIterator implements IteratorWithLowerBound { private final SSTableReader sstable; - private final ClusteringIndexFilter filter; + private final Slices slices; + private final boolean isReverseOrder; private final ColumnFilter selectedColumns; private final SSTableReadsListener listener; - private ClusteringBound lowerBound; + private Unfiltered lowerBoundMarker; private boolean firstItemRetrieved; public UnfilteredRowIteratorWithLowerBound(DecoratedKey partitionKey, @@ -59,26 +59,41 @@ public UnfilteredRowIteratorWithLowerBound(DecoratedKey partitionKey, ClusteringIndexFilter filter, ColumnFilter selectedColumns, SSTableReadsListener listener) + { + this(partitionKey, sstable, filter.getSlices(sstable.metadata()), filter.isReversed(), selectedColumns, listener); + } + + public UnfilteredRowIteratorWithLowerBound(DecoratedKey partitionKey, + SSTableReader sstable, + Slices slices, + boolean isReverseOrder, + ColumnFilter selectedColumns, + SSTableReadsListener listener) { super(partitionKey); this.sstable = sstable; - this.filter = filter; + this.slices = slices; + this.isReverseOrder = isReverseOrder; this.selectedColumns = selectedColumns; this.listener = listener; - this.lowerBound = null; this.firstItemRetrieved = false; } public Unfiltered lowerBound() { - if (lowerBound != null) - return makeBound(lowerBound); + if (lowerBoundMarker != null) + return lowerBoundMarker; // The partition index lower bound is more accurate than the sstable metadata lower bound but it is only - // present if the iterator has already been initialized, which we only do when there are tombstones since in - // this case we cannot use the sstable metadata clustering values - ClusteringBound ret = getPartitionIndexLowerBound(); - return ret != null ? makeBound(ret) : makeBound(getMetadataLowerBound()); + // present if the iterator has already been initialized + ClusteringBound lowerBound = getKeyCacheLowerBound(); + + if (lowerBound == null && canUseMetadataLowerBound()) + // If we coudn't get the lower bound from key cache, we try with metadata + lowerBound = getMetadataLowerBound(); + + lowerBoundMarker = makeBound(lowerBound); + return lowerBoundMarker; } private Unfiltered makeBound(ClusteringBound bound) @@ -86,10 +101,7 @@ private Unfiltered makeBound(ClusteringBound bound) if (bound == null) return null; - if (lowerBound != bound) - lowerBound = bound; - - return new RangeTombstoneBoundMarker(lowerBound, DeletionTime.LIVE); + return new ArtificialBoundMarker(bound); } @Override @@ -97,7 +109,7 @@ protected UnfilteredRowIterator initializeIterator() { @SuppressWarnings("resource") // 'iter' is added to iterators which is closed on exception, or through the closing of the final merged iterator UnfilteredRowIterator iter = RTBoundValidator.validate( - sstable.iterator(partitionKey(), filter.getSlices(metadata()), selectedColumns, filter.isReversed(), listener), + sstable.iterator(partitionKey(), slices, selectedColumns, isReverseOrder, listener), RTBoundValidator.Stage.SSTABLE, false ); @@ -113,10 +125,10 @@ protected Unfiltered computeNext() // Check that the lower bound is not bigger than the first item retrieved firstItemRetrieved = true; - if (lowerBound != null && ret != null) - assert comparator().compare(lowerBound, ret.clustering()) <= 0 + if (lowerBoundMarker != null && ret != null) + assert comparator().compare(lowerBoundMarker.clustering(), ret.clustering()) <= 0 : String.format("Lower bound [%s ]is bigger than first returned value [%s] for sstable %s", - lowerBound.toString(metadata()), + lowerBoundMarker.clustering().toString(metadata()), ret.toString(metadata()), sstable.getFilename()); @@ -125,7 +137,7 @@ assert comparator().compare(lowerBound, ret.clustering()) <= 0 private Comparator comparator() { - return filter.isReversed() ? metadata().comparator.reversed() : metadata().comparator; + return isReverseOrder ? metadata().comparator.reversed() : metadata().comparator; } @Override @@ -137,7 +149,7 @@ public TableMetadata metadata() @Override public boolean isReverseOrder() { - return filter.isReversed(); + return isReverseOrder; } @Override @@ -155,7 +167,7 @@ public EncodingStats stats() @Override public DeletionTime partitionLevelDeletion() { - if (!sstable.mayHaveTombstones()) + if (!sstable.getSSTableMetadata().hasPartitionLevelDeletions) return DeletionTime.LIVE; return super.partitionLevelDeletion(); @@ -170,39 +182,34 @@ public Row staticRow() return super.staticRow(); } - private static ClusteringBound createInclusiveOpen(boolean isReversed, ClusteringPrefix from) + private static ClusteringBound createArtificialLowerBound(boolean isReversed, ClusteringPrefix from) { - return from.accessor().factory().inclusiveOpen(isReversed, from.getRawValues()); + return !isReversed + ? from.accessor().factory().inclusiveOpen(false, from.getRawValues()).artificialLowerBound() + : from.accessor().factory().inclusiveOpen(true, from.getRawValues()).artificialUpperBound(); } /** * @return the lower bound stored on the index entry for this partition, if available. */ - private ClusteringBound getPartitionIndexLowerBound() + private ClusteringBound getKeyCacheLowerBound() { - // NOTE: CASSANDRA-11206 removed the lookup against the key-cache as the IndexInfo objects are no longer - // in memory for not heap backed IndexInfo objects (so, these are on disk). - // CASSANDRA-11369 is there to fix this afterwards. - - // Creating the iterator ensures that rowIndexEntry is loaded if available (partitions bigger than - // DatabaseDescriptor.column_index_size_in_kb) - if (!canUseMetadataLowerBound()) - maybeInit(); - - RowIndexEntry rowIndexEntry = sstable.getCachedPosition(partitionKey(), false); + BigTableRowIndexEntry rowIndexEntry = sstable.getCachedPosition(partitionKey(), false); if (rowIndexEntry == null || !rowIndexEntry.indexOnHeap()) return null; - try (RowIndexEntry.IndexInfoRetriever onHeapRetriever = rowIndexEntry.openWithIndex(null)) + try (BigTableRowIndexEntry.IndexInfoRetriever onHeapRetriever = rowIndexEntry.openWithIndex(null)) { - IndexInfo column = onHeapRetriever.columnsIndex(filter.isReversed() ? rowIndexEntry.columnsIndexCount() - 1 : 0); - ClusteringPrefix lowerBoundPrefix = filter.isReversed() ? column.lastName : column.firstName; + IndexInfo column = onHeapRetriever.columnsIndex(isReverseOrder ? rowIndexEntry.columnsIndexCount() - 1 : 0); + ClusteringPrefix lowerBoundPrefix = isReverseOrder ? column.lastName : column.firstName; + assert lowerBoundPrefix.getRawValues().length <= metadata().comparator.size() : String.format("Unexpected number of clustering values %d, expected %d or fewer for %s", lowerBoundPrefix.getRawValues().length, metadata().comparator.size(), sstable.getFilename()); - return createInclusiveOpen(filter.isReversed(), lowerBoundPrefix); + + return createArtificialLowerBound(isReverseOrder, lowerBoundPrefix); } catch (IOException e) { @@ -235,28 +242,39 @@ private ClusteringBound getPartitionIndexLowerBound() */ private boolean canUseMetadataLowerBound() { - // Side-note: pre-2.1 sstable stat file had clustering value arrays whose size may not match the comparator size - // and that would break getMetadataLowerBound. We don't support upgrade from 2.0 to 3.0 directly however so it's - // not a true concern. Besides, !sstable.mayHaveTombstones already ensure this is a 3.0 sstable anyway. - return !sstable.mayHaveTombstones() && !sstable.metadata().isCompactTable(); + if (sstable.metadata().isCompactTable()) + return false; + + Slices requestedSlices = slices; + + if (requestedSlices.isEmpty()) + return true; + + if (!isReverseOrder()) + { + return !requestedSlices.hasLowerBound() || + metadata().comparator.compare(requestedSlices.start(), sstable.getSSTableMetadata().coveredClustering.start()) < 0; + } + else + { + return !requestedSlices.hasUpperBound() || + metadata().comparator.compare(requestedSlices.end(), sstable.getSSTableMetadata().coveredClustering.end()) > 0; + } } /** * @return a global lower bound made from the clustering values stored in the sstable metadata, note that * this currently does not correctly compare tombstone bounds, especially ranges. */ - private ClusteringBound getMetadataLowerBound() + private @Nonnull ClusteringBound getMetadataLowerBound() { - if (!canUseMetadataLowerBound()) - return null; - final StatsMetadata m = sstable.getSSTableMetadata(); - List vals = filter.isReversed() ? m.maxClusteringValues : m.minClusteringValues; - assert vals.size() <= metadata().comparator.size() : + ClusteringBound bound = m.coveredClustering.open(isReverseOrder); + assert bound.size() <= metadata().comparator.size() : String.format("Unexpected number of clustering values %d, expected %d or fewer for %s", - vals.size(), + bound.size(), metadata().comparator.size(), sstable.getFilename()); - return ByteBufferAccessor.instance.factory().inclusiveOpen(filter.isReversed(), vals.toArray(new ByteBuffer[vals.size()])); + return !isReverseOrder ? bound.artificialLowerBound() : bound.artificialUpperBound(); } } diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java index 2eb5d8fde7bd..a38ef7d9f139 100644 --- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java +++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java @@ -594,7 +594,7 @@ protected Unfiltered getReduced() } } - protected void onKeyChange() + public void onKeyChange() { if (nextKind == Unfiltered.Kind.ROW) rowMerger.clear(); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java index 0bfe99311246..18e408ad48f9 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java @@ -34,7 +34,7 @@ import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableFormat; -import org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter; +import org.apache.cassandra.io.sstable.format.SSTableZeroCopyWriter; import org.apache.cassandra.io.sstable.metadata.StatsMetadata; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.schema.TableId; @@ -61,7 +61,7 @@ public class CassandraEntireSSTableStreamReader implements IStreamReader public CassandraEntireSSTableStreamReader(StreamMessageHeader messageHeader, CassandraStreamHeader streamHeader, StreamSession session) { - if (streamHeader.format != SSTableFormat.Type.BIG) + if (streamHeader.format != SSTableFormat.Type.BIG && streamHeader.format != SSTableFormat.Type.BTI) throw new AssertionError("Unsupported SSTable format " + streamHeader.format); if (session.getPendingRepair() != null) @@ -104,7 +104,7 @@ public SSTableMultiWriter read(DataInputPlus in) throws IOException prettyPrintMemory(totalSize), cfs.metadata()); - BigTableZeroCopyWriter writer = null; + SSTableZeroCopyWriter writer = null; try { @@ -167,7 +167,7 @@ private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOExceptio } @SuppressWarnings("resource") - protected BigTableZeroCopyWriter createWriter(ColumnFamilyStore cfs, long totalSize, Collection components) throws IOException + protected SSTableZeroCopyWriter createWriter(ColumnFamilyStore cfs, long totalSize, Collection components) throws IOException { File dataDir = getDataDir(cfs, totalSize); @@ -180,6 +180,6 @@ protected BigTableZeroCopyWriter createWriter(ColumnFamilyStore cfs, long totalS logger.debug("[Table #{}] {} Components to write: {}", cfs.metadata(), desc.filenameFor(Component.DATA), components); - return new BigTableZeroCopyWriter(desc, cfs.metadata, lifecycleNewTracker, components); + return new SSTableZeroCopyWriter(desc, cfs.metadata, lifecycleNewTracker, components); } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 0904720a627b..0e089c9d0096 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -43,7 +43,7 @@ */ public class CassandraOutgoingFile implements OutgoingStream { - private final Ref ref; + private final Ref ref; private final long estimatedKeys; private final List sections; private final String filename; @@ -51,7 +51,7 @@ public class CassandraOutgoingFile implements OutgoingStream private final StreamOperation operation; private final CassandraStreamHeader header; - public CassandraOutgoingFile(StreamOperation operation, Ref ref, + public CassandraOutgoingFile(StreamOperation operation, Ref ref, List sections, List> normalizedRanges, long estimatedKeys) { @@ -106,7 +106,7 @@ public static CassandraOutgoingFile fromStream(OutgoingStream stream) } @VisibleForTesting - public Ref getRef() + public Ref getRef() { return ref; } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index c9e10cf6a4cb..252b6c75c997 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -201,8 +201,9 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws I @VisibleForTesting public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function partitionerMapper) throws IOException { - Version sstableVersion = SSTableFormat.Type.current().info.getVersion(in.readUTF()); + String sstableVersionString = in.readUTF(); SSTableFormat.Type format = SSTableFormat.Type.validate(in.readUTF()); + Version sstableVersion = format.info.getVersion(sstableVersionString); long estimatedKeys = in.readLong(); int count = in.readInt(); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java index a84fd2764b55..b6c71fa7f08d 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java @@ -30,6 +30,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.big.BigTableReader; import org.apache.cassandra.locator.RangesAtEndpoint; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.service.ActiveRepairService; @@ -81,6 +82,7 @@ public StreamReceiver createStreamReceiver(StreamSession session, int totalStrea return new CassandraStreamReceiver(cfs, session, totalStreams); } + @SuppressWarnings("resource") // references placed onto returned collection or closed on error @Override public Collection createOutgoingStreams(StreamSession session, RangesAtEndpoint replicas, UUID pendingRepair, PreviewKind previewKind) { @@ -126,6 +128,8 @@ else if (pendingRepair == ActiveRepairService.NO_PENDING_REPAIR) return sstables; }).refs); + // Persistent memtables will not flush, make an sstable with their data. + cfs.writeAndAddMemtableRanges(session.getPendingRepair(), () -> Range.normalize(keyRanges), refs); List> normalizedFullRanges = Range.normalize(replicas.onlyFull().ranges()); List> normalizedAllRanges = Range.normalize(replicas.ranges()); @@ -134,9 +138,9 @@ else if (pendingRepair == ActiveRepairService.NO_PENDING_REPAIR) for (SSTableReader sstable : refs) { List> ranges = sstable.isRepaired() ? normalizedFullRanges : normalizedAllRanges; - List sections = sstable.getPositionsForRanges(ranges); + List sections = sstable.getPositionsForRanges(ranges); - Ref ref = refs.get(sstable); + Ref ref = refs.get(sstable); if (sections.isEmpty()) { ref.release(); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index b2b2ce5cf093..5749156036a3 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -28,7 +28,7 @@ import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.io.sstable.SSTable; -import org.apache.cassandra.streaming.StreamReceiveTask; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -182,7 +182,7 @@ private boolean hasCDC(ColumnFamilyStore cfs) * can be archived by the CDC process on discard. */ private boolean requiresWritePath(ColumnFamilyStore cfs) { - return hasCDC(cfs) || (session.streamOperation().requiresViewBuild() && hasViews(cfs)); + return hasCDC(cfs) || cfs.streamToMemtable() || (session.streamOperation().requiresViewBuild() && hasViews(cfs)); } private void sendThroughWritePath(ColumnFamilyStore cfs, Collection readers) { @@ -273,7 +273,7 @@ public void cleanup() // the streamed sstables. if (requiresWritePath) { - cfs.forceBlockingFlush(); + cfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.STREAMS_RECEIVED); abort(); } } diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentContext.java b/src/java/org/apache/cassandra/db/streaming/ComponentContext.java index b9c60b9f795e..49a9cf304307 100644 --- a/src/java/org/apache/cassandra/db/streaming/ComponentContext.java +++ b/src/java/org/apache/cassandra/db/streaming/ComponentContext.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.streaming; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +58,7 @@ public static ComponentContext create(Descriptor descriptor) { Map hardLinks = new HashMap<>(1); - for (Component component : MUTABLE_COMPONENTS) + for (Component component : Sets.intersection(MUTABLE_COMPONENTS, descriptor.getFormat().supportedComponents())) { File file = new File(descriptor.filenameFor(component)); if (!file.exists()) diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java index bb896caffa06..f823fb93ace1 100644 --- a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java +++ b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java @@ -37,9 +37,9 @@ */ public final class ComponentManifest implements Iterable { - private static final List STREAM_COMPONENTS = ImmutableList.of(Component.DATA, Component.PRIMARY_INDEX, Component.STATS, - Component.COMPRESSION_INFO, Component.FILTER, Component.SUMMARY, - Component.DIGEST, Component.CRC); + private static final List STREAM_COMPONENTS = ImmutableList.of(Component.DATA, Component.PRIMARY_INDEX, Component.PARTITION_INDEX, Component.ROW_INDEX, + Component.STATS, Component.COMPRESSION_INFO, Component.FILTER, Component.SUMMARY, + Component.DIGEST, Component.CRC); private final LinkedHashMap components; @@ -51,15 +51,18 @@ public ComponentManifest(Map components) @VisibleForTesting public static ComponentManifest create(Descriptor descriptor) { - LinkedHashMap components = new LinkedHashMap<>(STREAM_COMPONENTS.size()); + LinkedHashMap components = new LinkedHashMap<>(descriptor.getFormat().supportedComponents().size()); for (Component component : STREAM_COMPONENTS) { - File file = new File(descriptor.filenameFor(component)); - if (!file.exists()) - continue; + if (descriptor.getFormat().supportedComponents().contains(component)) + { + File file = new File(descriptor.filenameFor(component)); + if (!file.exists()) + continue; - components.put(component, file.length()); + components.put(component, file.length()); + } } return new ComponentManifest(components); diff --git a/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java b/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java new file mode 100644 index 000000000000..31b65c24f72d --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/CollectionMergeTrie.java @@ -0,0 +1,264 @@ +/* + * 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.tries; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; + +import com.google.common.collect.Iterables; + +/** + * A merged view of multiple tries. + * + * Note: We use same input and output types to be able to switch to directly returning single-origin branches. + */ +class CollectionMergeTrie extends Trie +{ + private final CollectionMergeResolver resolver; // only called on more than one input + protected final Collection> inputs; + + CollectionMergeTrie(Collection> inputs, CollectionMergeResolver resolver) + { + this.resolver = resolver; + this.inputs = inputs; + } + + @Override + protected Cursor cursor() + { + return new CollectionMergeCursor<>(resolver, inputs); + } + + static boolean greaterCursor(Cursor c1, Cursor c2) + { + int c1level = c1.level(); + int c2level = c2.level(); + if (c1level != c2level) + return c1level < c2level; + return c1.incomingTransition() > c2.incomingTransition(); + } + + static boolean equalCursor(Cursor c1, Cursor c2) + { + return c1.level() == c2.level() && c1.incomingTransition() == c2.incomingTransition(); + } + + static class CollectionMergeCursor implements Cursor + { + private final CollectionMergeResolver resolver; + private final Cursor[] heap; + private Cursor head; + private final List contents; + + public CollectionMergeCursor(CollectionMergeResolver resolver, Collection> inputs) + { + this.resolver = resolver; + int count = inputs.size(); + heap = new Cursor[count - 1]; + contents = new ArrayList<>(count); + int i = -1; + --count; + for (Trie trie : inputs) + { + Cursor cursor = trie.cursor(); + if (cursor.level() < 0 && count > 0) + heap[--count] = cursor; // empty trie / no root, put it at the end + else if (i >= 0) + heap[i++] = cursor; + else + { + head = cursor; + ++i; + } + } + } + + @Override + public int advance() + { + advance(0); + return maybeSwapHead(head.advance()); + } + + /** + * Advance the state of the input at the given index and any of its descendants that are at the same + * transition byte and restore the heap invariant for the subtree rooted at the given index. + * Calls itself recursively and used by advance with index = 0 to advance the state of the merge. + */ + private void advance(int index) + { + if (index >= heap.length) + return; + Cursor item = heap[index]; + if (!equalCursor(item, head)) + return; + + // If the children are at the same transition byte, they also need advancing and their subheap + // invariant to be restored. + advance(index * 2 + 1); + advance(index * 2 + 2); + + item.advance(); + + // At this point the heaps at both children are advanced and well-formed. Place current node in its + // proper position. + heapifyDown(item, index); + // The heap rooted at index is now advanced and well-formed. + } + + /** + * Push the given state down in the heap from the given index until it finds its proper place among + * the subheap rooted at that position. + */ + private void heapifyDown(Cursor item, int index) + { + while (true) + { + int next = index * 2 + 1; + if (next >= heap.length) + break; + // Select the smaller of the two children to push down to. + if (next + 1 < heap.length && greaterCursor(heap[next], heap[next + 1])) + ++next; + // If the child is greater or equal, the invariant has been restored. + if (!greaterCursor(item, heap[next])) + break; + heap[index] = heap[next]; + index = next; + } + heap[index] = item; + } + + private int maybeSwapHead(int headLevel) + { + int heap0Level = heap[0].level(); + if (headLevel > heap0Level || + (headLevel == heap0Level && head.incomingTransition() <= heap[0].incomingTransition())) + return headLevel; + // otherwise we need to swap heap and heap[0] + Cursor newHeap0 = head; + head = heap[0]; + heapifyDown(newHeap0, 0); + return heap0Level; + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + if (equalCursor(heap[0], head)) + return advance(); // more than one source at current position, can't do multiple. + + return maybeSwapHead(head.advanceMultiple(receiver)); + } + + @Override + public int ascend() + { + ascend(0); + return maybeSwapHead(head.ascend()); + } + + private void ascend(int index) + { + if (index >= heap.length) + return; + Cursor item = heap[index]; + if (head.level() != item.level()) + return; + + ascend(index * 2 + 1); + ascend(index * 2 + 2); + + item.ascend(); + heapifyDown(item, index); + } + + @Override + public int level() + { + return head.level(); + } + + @Override + public int incomingTransition() + { + return head.incomingTransition(); + } + + @Override + public T content() + { + T itemContent = head.content(); + if (itemContent != null) + contents.add(itemContent); + + collectContent(0); + T toReturn; + switch (contents.size()) + { + case 0: + toReturn = null; + break; + case 1: + toReturn = contents.get(0); + break; + default: + toReturn = resolver.resolve(contents); + break; + } + contents.clear(); + return toReturn; + } + + private void collectContent(int index) + { + if (index >= heap.length) + return; + Cursor item = heap[index]; + if (!equalCursor(item, head)) + return; + + T itemContent = item.content(); + if (itemContent != null) + contents.add(itemContent); + + collectContent(index * 2 + 1); + collectContent(index * 2 + 2); + } + } + + /** + * Special instance for sources that are guaranteed distinct. The main difference is that we can form unordered + * value list by concatenating sources. + */ + static class Distinct extends CollectionMergeTrie + { + Distinct(Collection> inputs) + { + super(inputs, throwingResolver()); + } + + @Override + public Iterable valuesUnordered() + { + return Iterables.concat(Iterables.transform(inputs, Trie::valuesUnordered)); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java b/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java new file mode 100644 index 000000000000..1a0a65eb59b1 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableReadTrie.java @@ -0,0 +1,869 @@ +/* + * 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.tries; + +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.function.Function; + +import org.agrona.concurrent.UnsafeBuffer; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * Memtable trie, i.e. an in-memory trie built for fast modification and reads executing concurrently with writes from + * a single mutator thread. + * + * This class provides the read-only functionality, expanded in {@link MemtableTrie} to writes. + */ +public class MemtableReadTrie extends Trie +{ + /* + TRIE FORMAT AND NODE TYPES + + The memtable trie uses five different types of nodes: + - "leaf" nodes, which have content and no children; + - single-transition "chain" nodes, which have exactly one child; while each node is a single transition, they are + called "chain" because multiple such transition are packed in a block. + - "sparse" nodes which have between two and six children; + - "split" nodes for anything above six children; + - "prefix" nodes that augment one of the other types (except leaf) with content. + + The data for all nodes except leaf ones is stored in a contiguous 'node buffer' and laid out in blocks of 32 bytes. + A block only contains data for a single type of node, but there is no direct correspondence between block and node + in that: + - a single block can contain multiple "chain" nodes. + - a sparse node occupies exactly one block. + - a split node occupies a variable number of blocks. + - a prefix node can be placed in the same block as the node it augments, or in a separate block. + + Nodes are referenced in that buffer by an integer position/pointer, the 'node pointer'. Note that node pointers are + not pointing at the beginning of blocks, and we call 'pointer offset' the offset of the node pointer to the block it + points into. The value of a 'node pointer' is used to decide what kind of node is pointed: + + - If the pointer is negative, we have a leaf node. Since a leaf has no children, we need no data outside of its + content to represent it, and that content is stored in a 'content list', not in the nodes buffer. The content + of a particular leaf node is located at the ~pointer position in the content list (~ instead of - so that -1 can + correspond to position 0). + + - If the 'pointer offset' is smaller than 28, we have a chain node with one transition. The transition character is + the byte at the position pointed in the 'node buffer', and the child is pointed by: + - the integer value at offset 28 of the block pointed if the 'pointer offset' is 27 + - pointer + 1 (which is guaranteed to have offset smaller than 28, i.e. to be a chain node), otherwise + In other words, a chain block contains a sequence of characters that leads to the child whose address is at + offset 28. It may have between 1 and 28 characters depending on the pointer with which the block is entered. + + - If the 'pointer offset' is 30, we have a sparse node. The data of a sparse node occupies a full block and is laid + out as: + - six pointers to children at offsets 0 to 24 + - six transition characters at offsets 24 to 30 + - an order word stored in the two bytes at offset 30 + To enable in-place addition of children, the pointers and transition characters are not stored ordered. + Instead, we use an order encoding in the last 2 bytes of the node. The encoding is a base-6 number which + describes the order of the transitions (least significant digit being the smallest). + The node must have at least two transitions and the transition at position 0 is never the biggest (we can + enforce this by choosing for position 0 the smaller of the two transitions a sparse node starts with). This + allows iteration over the order word (which divides said word by 6 each step) to finish when the result becomes 0. + + - If the 'pointer offset' is 28, the node is a split one. Split nodes are dense, meaning that there is a direct + mapping between a transition character and the address of the associated pointer, and new children can easily be + added in place. + Split nodes occupy multiple blocks, and a child is located by traversing 3 layers of pointers: + - the first pointer is within the top-level block (the one pointed by the pointer) and points to a "mid" block. + The top-level block has 4 such pointers to "mid" block, located between offset 16 and 32. + - the 2nd pointer is within the "mid" block and points to a "tail" block. A "mid" block has 8 such pointers + occupying the whole block. + - the 3rd pointer is with the "tail" block and is the actual child pointer. Like "mid" block, there are 8 such + pointers (so we finally address 4 * 8 * 8 = 256 children). + To find a child, we thus need to know the index of the pointer to follow within the top-level block, the index + of the one in the "mid" block and the index in the "tail" block. For that, we split the transition byte in a + sequence of 2-3-3 bits: + - the first 2 bits are the index in the top-level block; + - the next 3 bits, the index in the "mid" block; + - and the last 3 bits the index in the "tail" block. + This layout allows the node to use the smaller fixed-size blocks (instead of 256*4 bytes for the whole character + space) and also leaves some room in the head block (the 16 first bytes) for additional information (which we can + use to store prefix nodes containing things like deletion times). + One split node may need up to 1 + 4 + 4*8 blocks (1184 bytes) to store all its children. + + - If the pointer offset is 31, we have a prefix node. These are two types: + -- Embedded prefix nodes occupy the free bytes in a chain or split node. The byte at offset 4 has the offset + within the 32-byte block for the augmented node. + -- Full prefix nodes have 0xFF at offset 4 and a pointer at 28, pointing to the augmented node. + Both types contain an index for content at offset 0. The augmented node cannot be a leaf or NONE -- in the former + case the leaf itself contains the content index, in the latter we use a leaf instead. + The term "node" when applied to these is a bit of a misnomer as they are not presented as separate nodes during + traversals. Instead, they augment a node, changing only its content. Internally we create a Node object for the + augmented node and wrap a PrefixNode around it, which changes the `content()` method and routes all other + calls to the augmented node's methods. + + When building a trie we first allocate the content, then create a chain node leading to it. While we only have + single transitions leading to a chain node, we can expand that node (attaching a character and using pointer - 1) + instead of creating a new one. When a chain node already has a child and needs a new one added we change the type + (i.e. create a new node and remap the parent) to sparse with two children. When a six-child sparse node needs a new + child, we switch to split. + + Blocks currently are not reused, because we do not yet have a mechanism to tell when readers are done with blocks + they are referencing. This currently causes a very low overhead (because we change data in place with the only + exception of nodes needing to change type) and is planned to be addressed later. + + For an example of the evolution of the trie, see MemtableTrie.md. + */ + + static final int BLOCK_SIZE = 32; + + // Biggest block offset that can contain a pointer. + static final int LAST_POINTER_OFFSET = BLOCK_SIZE - 4; + + /* + Block offsets used to identify node types (by comparing them to the node 'pointer offset'). + */ + + // split node (dense, 2-3-3 transitions), laid out as 4 pointers to "mid" block, with has 8 pointers to "tail" block, + // which has 8 pointers to children + static final int SPLIT_OFFSET = BLOCK_SIZE - 4; + // sparse node, unordered list of up to 6 transition, laid out as 6 transition pointers followed by 6 transition + // bytes. The last two bytes contain an ordering of the transitions (in base-6) which is used for iteration. On + // update the pointer is set last, i.e. during reads the node may show that a transition exists and list a character + // for it, but pointer may still be null. + static final int SPARSE_OFFSET = BLOCK_SIZE - 2; + // min and max offset for a chain node. A block of chain node is laid out as a pointer at LAST_POINTER_OFFSET, + // preceded by characters that lead to it. Thus a full chain block contains BLOCK_SIZE-4 transitions/chain nodes. + static final int CHAIN_MIN_OFFSET = 0; + static final int CHAIN_MAX_OFFSET = BLOCK_SIZE - 5; + // Prefix node, an intermediate node augmenting its child node with content. + static final int PREFIX_OFFSET = BLOCK_SIZE - 1; + + /* + Offsets and values for navigating in a block for particular node type. Those offsets are 'from the node pointer' + (not the block start) and can be thus negative since node pointers points towards the end of blocks. + */ + + // Offset to the first pointer (to "mid" blocks) of a split node. + static final int SPLIT_POINTER_OFFSET = 16 - SPLIT_OFFSET; + + static final int SPARSE_CHILD_COUNT = 6; + // Offset to the first child pointer of a spare node (laid out from the start of the block) + static final int SPARSE_CHILDREN_OFFSET = 0 - SPARSE_OFFSET; + // Offset to the first transition byte of a sparse node (laid out after the child pointers) + static final int SPARSE_BYTES_OFFSET = SPARSE_CHILD_COUNT * 4 - SPARSE_OFFSET; + // Offset to the order word of a sparse node (laid out after the children (pointer + transition byte)) + static final int SPARSE_ORDER_OFFSET = SPARSE_CHILD_COUNT * 5 - SPARSE_OFFSET; // 0 + + // Offset of the flag byte in a prefix node. In shared blocks, this contains the offset of the next node. + static final int PREFIX_FLAGS_OFFSET = 4 - PREFIX_OFFSET; + // Offset of the content id + static final int PREFIX_CONTENT_OFFSET = 0 - PREFIX_OFFSET; + // Offset of the next pointer in a non-shared prefix node + static final int PREFIX_POINTER_OFFSET = LAST_POINTER_OFFSET - PREFIX_OFFSET; + + // Initial capacity for the node data buffer. + static final int INITIAL_BUFFER_CAPACITY = 256; + + /** + * Value used as null for node pointers. + * No node can use this address (we enforce this by not allowing chain nodes to grow to position 0). + * Do not change this as the code relies there being a NONE placed in all bytes of the block that are not set. + */ + static final int NONE = 0; + + volatile int root; + + /* + EXPANDABLE DATA STORAGE + + The tries will need more and more space in buffers and content lists as they grow. Instead of using ArrayList-like + reallocation with copying, which may be prohibitively expensive for large buffers, we use a sequence of + buffers/content arrays that double in size on every expansion. + + For a given address x the index of the buffer can be found with the following calculation: + index_of_most_significant_set_bit(x / min_size + 1) + (relying on sum (2^i) for i in [0, n-1] == 2^n - 1) which can be performed quickly on modern hardware. + + Finding the offset within the buffer is then + x + min - (min << buffer_index) + + The allocated space starts 256 bytes for the buffer and 16 entries for the content list. + + Note that a buffer is not allowed to split 32-byte blocks (code assumes same buffer can be used for all bytes + inside the block). + + TODO: implement delay and retry on space hitting the 2GB barrier. + */ + + static final int BUF_START_SHIFT = 8; + static final int BUF_START_SIZE = 1 << BUF_START_SHIFT; + + static final int CONTENTS_START_SHIFT = 4; + static final int CONTENTS_START_SIZE = 1 << CONTENTS_START_SHIFT; + + final UnsafeBuffer[] buffers; + final AtomicReferenceArray[] contentArrays; + + MemtableReadTrie(UnsafeBuffer[] buffers, AtomicReferenceArray[] contentArrays, int root) + { + this.buffers = buffers; + this.contentArrays = contentArrays; + this.root = root; + } + + /* + Buffer, content list and block management + */ + int getChunkIdx(int pos, int minChunkShift, int minChunkSize) + { + return 31 - minChunkShift - Integer.numberOfLeadingZeros(pos + minChunkSize); + } + + int getChunkOffset(int pos, int chunkIndex, int minChunkSize) + { + return pos + minChunkSize - (minChunkSize << chunkIndex); + } + + UnsafeBuffer getBuffer(int pos) + { + int leadBit = getChunkIdx(pos, BUF_START_SHIFT, BUF_START_SIZE); + return buffers[leadBit]; + } + + int getOffset(int pos) + { + int leadBit = getChunkIdx(pos, BUF_START_SHIFT, BUF_START_SIZE); + return getChunkOffset(pos, leadBit, BUF_START_SIZE); + } + + + /** + * Pointer offset for a node pointer. + */ + int offset(int pos) + { + return pos & (BLOCK_SIZE - 1); + } + + final int getByte(int pos) + { + return getBuffer(pos).getByte(getOffset(pos)) & 0xFF; + } + + final int getShort(int pos) + { + return getBuffer(pos).getShort(getOffset(pos)) & 0xFFFF; + } + + final int getInt(int pos) + { + return getBuffer(pos).getInt(getOffset(pos)); + } + + T getContent(int index) + { + int leadBit = getChunkIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); + int ofs = getChunkOffset(index, leadBit, CONTENTS_START_SIZE); + AtomicReferenceArray array = contentArrays[leadBit]; + return array.get(ofs); + } + + /* + Reading node content + */ + + boolean isNull(int node) + { + return node == NONE; + } + + boolean isLeaf(int node) + { + return node < NONE; + } + + boolean isNullOrLeaf(int node) + { + return node <= NONE; + } + + /** + * Returns the child pointer of a chain-block (that is, the point to the child of the last node of said + * chain-block). + */ + private int chainBlockChildPointer(int node) + { + return (node & -BLOCK_SIZE) | LAST_POINTER_OFFSET; + } + + /** + * Get a node's child for the given transition character + */ + int getChild(int node, int trans) + { + if (isNullOrLeaf(node)) + return NONE; + + node = followContentTransition(node); + + switch (offset(node)) + { + case SPARSE_OFFSET: + return getSparseChild(node, trans); + case SPLIT_OFFSET: + return getSplitChild(node, trans); + case CHAIN_MAX_OFFSET: + if (trans != getByte(node)) + return NONE; + return getInt(node + 1); + default: + if (trans != getByte(node)) + return NONE; + return node + 1; + } + } + + protected int followContentTransition(int node) + { + if (isNullOrLeaf(node)) + return NONE; + + if (offset(node) == PREFIX_OFFSET) + { + int b = getByte(node + PREFIX_FLAGS_OFFSET); + if (b < BLOCK_SIZE) + node = node - PREFIX_OFFSET + b; + else + node = getInt(node + PREFIX_POINTER_OFFSET); + + assert node >= 0 && offset(node) != PREFIX_OFFSET; + } + return node; + } + + /** + * Advance as long as the cell pointed to by the given pointer will let you. + *

    + * This is the same as getChild(node, first), except for chain nodes where it would walk the fill chain as long as + * the input source matches. + */ + int advance(int node, int first, ByteSource rest) + { + if (isNullOrLeaf(node)) + return NONE; + + node = followContentTransition(node); + + switch (offset(node)) + { + case SPARSE_OFFSET: + return getSparseChild(node, first); + case SPLIT_OFFSET: + return getSplitChild(node, first); + default: + // Check the first byte matches the expected + if (getByte(node) != first) + return NONE; + // Check the rest of the bytes provided by the chain node (limit - node - 1 many) + int limit = chainBlockChildPointer(node); + while (++node < limit) + { + first = rest.next(); + if (getByte(node) != first) + return NONE; + } + // All bytes matched, follow the pointer + return getInt(limit); + } + } + + /** + * Get the child for the given transition character, knowing that the node is sparse + */ + int getSparseChild(int node, int trans) + { + for (int i = 0; i < SPARSE_CHILD_COUNT; ++i) + { + if (getByte(node + SPARSE_BYTES_OFFSET + i) == trans) + { + int child = getInt(node + SPARSE_CHILDREN_OFFSET + i * 4); + + // we can't trust the transition character read above, because it may have been fetched before a + // concurrent update happened, and the update may have managed to modify the pointer by now. + // However, if we read it now that we have accessed the volatile pointer, it must have the correct + // value as it is set before the pointer. + if (child != NONE && getByte(node + SPARSE_BYTES_OFFSET + i) == trans) + return child; + } + } + return NONE; + } + + /** + * Given a transition, returns the corresponding index (within the node block) of the pointer to the mid block of + * a split node. + */ + int splitNodeMidIndex(int trans) + { + // first 2 bytes of the 2-3-3 split + return (trans >> 6); + } + + /** + * Given a transition, returns the corresponding index (within the mid block) of the pointer to the tail block of + * a split node. + */ + int splitNodeTailIndex(int trans) + { + // second 3 bytes of the 2-3-3 split + return (trans >> 3) & 0x7; + } + + /** + * Given a transition, returns the corresponding index (within the tail block) of the pointer to the child of + * a split node. + */ + int splitNodeChildIndex(int trans) + { + // third 3 bytes of the 2-3-3 split + return trans & 0x7; + } + + /** + * Get the child for the given transition character, knowing that the node is split + */ + int getSplitChild(int node, int trans) + { + int mid = getInt(node + SPLIT_POINTER_OFFSET + splitNodeMidIndex(trans) * 4); + if (isNull(mid)) + return NONE; + + int tail = getInt(mid + splitNodeTailIndex(trans) * 4); + if (isNull(tail)) + return NONE; + return getInt(tail + splitNodeChildIndex(trans) * 4); + } + + /** + * Get the content for a given node + */ + T getNodeContent(int node) + { + if (isLeaf(node)) + return getContent(~node); + + if (offset(node) != PREFIX_OFFSET) + return null; + + int index = getInt(node + PREFIX_CONTENT_OFFSET); + return (index >= 0) + ? getContent(index) + : null; + } + + /* + Cursor implementation + */ + + class MemtableCursor implements Cursor + { + private int[] backtrack = new int[48]; + private int backtrackLevel = 0; + + private int currentNode; + + private int incomingTransition; + private T content; + private int level = -1; + + MemtableCursor() + { + descendInto(root, -1); + } + + private int node(int backtrackLevel) + { + return backtrack[backtrackLevel * 3 + 0]; + } + + private int data(int backtrackLevel) + { + return backtrack[backtrackLevel * 3 + 1]; + } + + private int level(int backtrackLevel) + { + return backtrack[backtrackLevel * 3 + 2]; + } + + void addBacktrack(int node, int data, int level) + { + if (backtrackLevel * 3 >= backtrack.length) + backtrack = Arrays.copyOf(backtrack, backtrack.length * 2); + backtrack[backtrackLevel * 3 + 0] = node; + backtrack[backtrackLevel * 3 + 1] = data; + backtrack[backtrackLevel * 3 + 2] = level; + ++backtrackLevel; + } + + @Override + public int advance() + { + return advance(0); + } + + private int advance(int data) + { + while (true) + { + if (advanceToNextChild(currentNode, data)) + return level; + + if (--backtrackLevel < 0) + return level = -1; + + level = level(backtrackLevel); + currentNode = node(backtrackLevel); + data = data(backtrackLevel); + } + } + + @Override + public int ascend() + { + if (--backtrackLevel < 0) + return level = -1; + + level = level(backtrackLevel); + currentNode = node(backtrackLevel); + int data = data(backtrackLevel); + return advance(data); + } + + private int descendInto(int child, int transition) + { + ++level; + incomingTransition = transition; + content = getNodeContent(child); + currentNode = followContentTransition(child); + return level; + } + + private int descendIntoChain(int child, int transition) + { + ++level; + incomingTransition = transition; + content = null; + currentNode = child; + return level; + } + + private boolean advanceToNextChild(int node, int data) + { + if (isNull(node)) + return false; + + switch (offset(node)) + { + case SPLIT_OFFSET: + return nextValidSplitTransition(node, data); + case SPARSE_OFFSET: + return nextValidSparseTransition(node, data); + default: + return getChainTransition(node); + } + } + + private boolean nextValidSplitTransition(int node, int trans) + { + assert trans >= 0 && trans <= 0xFF; + // To avoid repeatedly following the top transitions, we put backtracking entries for each level of the + // split sub-trie and use the bits of `trans` to understand which level the backtracking info points to. + + int childIndex = splitNodeChildIndex(trans); + if (childIndex == 0) + { + int tailIndex = splitNodeTailIndex(trans); + if (tailIndex == 0) + { + int midIndex = splitNodeMidIndex(trans); + int mid; + while (true) + { + mid = getInt(node + SPLIT_POINTER_OFFSET + midIndex * 4); + if (!isNull(mid)) + break; + if (++midIndex == 4) + return false; + } + if (midIndex + 1 < 4) + addBacktrack(node, (midIndex + 1) << 6, level); // Store backtracking pos for the top sub-node + trans = midIndex << 6; + node = mid + SPLIT_OFFSET; // Adjust sub-node pointer so that backtracking can bring us here + } + else + trans = trans & -(1 << 6); + + int tail; + while (true) + { + tail = getInt(node - SPLIT_OFFSET + tailIndex * 4); + if (!isNull(tail)) + break; + if (++tailIndex == 8) + return false; + } + if (tailIndex + 1 < 8) + addBacktrack(node, (tailIndex + 1) << 3 | trans, level); // Store backtracking pos for the mid sub-node + trans = tailIndex << 3 | trans; + node = tail + SPLIT_OFFSET; + } + else + trans = trans & -(1 << 3); + + int child; + while (true) + { + child = getInt(node - SPLIT_OFFSET + childIndex * 4); + if (!isNull(child)) + break; + if (++childIndex == 8) + return false; + } + if (childIndex + 1 < 8) + addBacktrack(node, (childIndex + 1) | trans, level); + trans = childIndex | trans; + descendInto(child, trans); + return true; + } + + private boolean nextValidSparseTransition(int node, int data) + { + UnsafeBuffer nodeBuffer = getBuffer(node); + int nodeOfs = getOffset(node); + if (data <= 0) + data = nodeBuffer.getShort(nodeOfs + SPARSE_ORDER_OFFSET) & 0xFFFF; + int index = data % SPARSE_CHILD_COUNT; + data = data / SPARSE_CHILD_COUNT; + if (data > 0) + addBacktrack(node, data, level); + int child = nodeBuffer.getInt(nodeOfs + SPARSE_CHILDREN_OFFSET + index * 4); + int transition = nodeBuffer.getByte(nodeOfs + SPARSE_BYTES_OFFSET + index) & 0xFF; + descendInto(child, transition); + return true; + } + + private boolean getChainTransition(int node) + { + // no backtracking needed + UnsafeBuffer nodeBuffer = getBuffer(node); + int nodeOfs = getOffset(node); + int transition = nodeBuffer.getByte(nodeOfs) & 0xFF; + int next = node + 1; + if (offset(next) <= CHAIN_MAX_OFFSET) + descendIntoChain(next, transition); + else + descendInto(nodeBuffer.getInt(nodeOfs + 1), transition); + return true; + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + int node = currentNode; + if (!isChainNode(node)) + return advance(); + + while (true) + { + UnsafeBuffer buffer = getBuffer(node); + int ofs = getOffset(node); + int pointer = chainBlockChildPointer(ofs); + int child = buffer.getInt(pointer); + int length = pointer - ofs; + if (isNullOrLeaf(child) || offset(child) == PREFIX_OFFSET) + { + --length; // leave the last byte for incomingTransition + if (receiver != null && length > 0) + receiver.add(buffer, ofs, length); + level += length; + + return descendInto(child, buffer.getByte(pointer - 1) & 0xFF); + } + + if (receiver != null) + receiver.add(buffer, ofs, length); + level += length; + + if (!isChainNode(child)) + { + boolean success = advanceToNextChild(child, 0); + assert success; + return level; + } + node = child; + } + } + + int advanceChainPath(TransitionsReceiver receiver) + { + int node = currentNode; + if (!isChainNode(node)) + return advance(); + + UnsafeBuffer buffer = getBuffer(node); + int ofs = getOffset(node); + int pointer = chainBlockChildPointer(ofs); + int child = buffer.getInt(pointer); + int length = pointer - ofs; + --length; // leave the last byte for incomingTransition + if (receiver != null && length > 0) + receiver.add(buffer, ofs, length); + level += length; + + return descendInto(child, buffer.getByte(pointer - 1) & 0xFF); + } + + public int level() + { + return level; + } + + public T content() + { + return content; + } + + public int incomingTransition() + { + return incomingTransition; + } + } + + private boolean isChainNode(int node) + { + return !isNullOrLeaf(node) && offset(node) <= CHAIN_MAX_OFFSET; + } + + public MemtableCursor cursor() + { + return new MemtableCursor(); + } + + /* + Direct read methods + */ + + /** + * Get the content mapped by the specified key. + * Fast implementation using integer node addresses. + */ + public T get(ByteComparable path) + { + int n = root; + ByteSource source = path.asComparableBytes(BYTE_COMPARABLE_VERSION); + while (!isNull(n)) + { + int c = source.next(); + if (c == ByteSource.END_OF_STREAM) + return getNodeContent(n); + + n = advance(n, c, source); + } + + return null; + } + + public boolean isEmpty() + { + return isNull(root); + } + + /** + * Override of dump to provide more detailed printout that includes the type of each node in the trie. + */ + @Override + public String dump(Function contentToString) + { + MemtableCursor source = cursor(); + class TypedNodesCursor implements Cursor + { + @Override + public int advance() + { + return source.advance(); + } + + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + return source.advanceChainPath(receiver); + } + + @Override + public int ascend() + { + return source.ascend(); + } + + @Override + public int level() + { + return source.level(); + } + + @Override + public int incomingTransition() + { + return source.incomingTransition(); + } + + @Override + public String content() + { + String type = null; + int node = source.currentNode; + if (!isNullOrLeaf(node)) + { + switch (offset(node)) + { + case SPARSE_OFFSET: + type = "[SPARSE]"; + break; + case SPLIT_OFFSET: + type = "[SPLIT]"; + break; + case PREFIX_OFFSET: + throw new AssertionError("Unexpected prefix as cursor currentNode."); + default: + type = "[CHAIN]"; + break; + } + } + T content = source.content(); + if (content != null) + { + if (type != null) + return contentToString.apply(content) + " -> " + type; + else + return contentToString.apply(content); + } + else + return type; + } + } + return TrieDumper.dump(Object::toString, new TypedNodesCursor()); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.java b/src/java/org/apache/cassandra/db/tries/MemtableTrie.java new file mode 100644 index 000000000000..8a1bfd6b5bd1 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.java @@ -0,0 +1,983 @@ +/* + * 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.tries; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicReferenceArray; + +import com.google.common.annotations.VisibleForTesting; + +import net.nicoulaj.compilecommand.annotations.DontInline; +import org.agrona.concurrent.UnsafeBuffer; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.ObjectSizes; +import org.github.jamm.MemoryLayoutSpecification; + +/** + * Memtable trie, i.e. an in-memory trie built for fast modification and reads executing concurrently with writes from + * a single mutator thread. + * + * Writes to this should be atomic (i.e. reads should see either the content before the write, or the content after the + * write; if any read sees the write, then any subsequent (i.e. started after it completed) read should also see it). + * This implementation does not currently guarantee this, but we still get the desired result as `apply` is only used + * with singleton tries. + */ +public class MemtableTrie extends MemtableReadTrie +{ + // See the trie format description in MemtableReadTrie. + + /** + * Trie size limit. This is not enforced, but users must check from time to time that it is not exceeded (using + * reachedAllocatedSizeThreshold()) and start switching to a new trie if it is. + * This must be done to avoid tries growing beyond their hard 2GB size limit (due to the 32-bit pointers). + */ + private static final int ALLOCATED_SIZE_THRESHOLD; + static + { + String propertyName = "cassandra.trie_size_limit_mb"; + // Default threshold + 10% == 1 GB. Adjusted slightly up to avoid a tiny final allocation for the 2G max. + int limitInMB = Integer.parseInt(System.getProperty(propertyName, + Integer.toString(1024 * 10 / 11 + 1))); + if (limitInMB < 1 || limitInMB > 2047) + throw new AssertionError(propertyName + " must be within 1 and 2047"); + ALLOCATED_SIZE_THRESHOLD = 1024 * 1024 * limitInMB; + } + + private int allocatedPos = 0; + private int contentCount = 0; + + private final BufferType bufferType; // on or off heap + + private static final long EMPTY_SIZE_ON_HEAP; // for space calculations + private static final long EMPTY_SIZE_OFF_HEAP; // for space calculations + + static + { + MemtableTrie empty = new MemtableTrie<>(BufferType.ON_HEAP); + EMPTY_SIZE_ON_HEAP = ObjectSizes.measureDeep(empty); + empty = new MemtableTrie<>(BufferType.OFF_HEAP); + EMPTY_SIZE_OFF_HEAP = ObjectSizes.measureDeep(empty); + } + + public MemtableTrie(BufferType bufferType) + { + super(new UnsafeBuffer[31 - BUF_START_SHIFT], // last one is 1G for a total of ~2G bytes + new AtomicReferenceArray[29 - CONTENTS_START_SHIFT], // takes at least 4 bytes to write pointer to one content -> 4 times smaller than buffers + NONE); + this.bufferType = bufferType; + assert INITIAL_BUFFER_CAPACITY % BLOCK_SIZE == 0; + } + + // Buffer, content list and block management + + public static class SpaceExhaustedException extends Exception + { + public SpaceExhaustedException() + { + super("The hard 2GB limit on trie size has been exceeded"); + } + } + + final void putInt(int pos, int value) + { + getBuffer(pos).putInt(getOffset(pos), value); + } + + final void putIntOrdered(int pos, int value) + { + getBuffer(pos).putIntOrdered(getOffset(pos), value); + } + + final void putIntVolatile(int pos, int value) + { + getBuffer(pos).putIntVolatile(getOffset(pos), value); + } + + final void putShort(int pos, short value) + { + getBuffer(pos).putShort(getOffset(pos), value); + } + + final void putShortVolatile(int pos, short value) + { + getBuffer(pos).putShort(getOffset(pos), value); + } + + final void putByte(int pos, byte value) + { + getBuffer(pos).putByte(getOffset(pos), value); + } + + + private int allocateBlock() throws SpaceExhaustedException + { + // Note: If this method is modified, please run MemtableTrieTest.testOver1GSize to verify it acts correctly + // close to the 2G limit. + int v = allocatedPos; + if (getOffset(v) == 0) + { + int leadBit = getChunkIdx(v, BUF_START_SHIFT, BUF_START_SIZE); + if (leadBit == 31) + throw new SpaceExhaustedException(); + + assert buffers[leadBit] == null; + ByteBuffer newBuffer = bufferType.allocate(BUF_START_SIZE << leadBit); + buffers[leadBit] = new UnsafeBuffer(newBuffer); + // The above does not contain any happens-before enforcing writes, thus at this point the new buffer may be + // invisible to any concurrent readers. Touching the volatile root pointer (which any new read must go + // through) enforces a happens-before that makes it visible to all new reads (note: when the write completes + // it must do some volatile write, but that will be in the new buffer and without the line below could + // remain unreachable by other cores). + root = root; + } + + allocatedPos += BLOCK_SIZE; + return v; + } + + private int addContent(T value) + { + int index = contentCount++; + int leadBit = getChunkIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); + int ofs = getChunkOffset(index, leadBit, CONTENTS_START_SIZE); + AtomicReferenceArray array = contentArrays[leadBit]; + if (array == null) + { + assert ofs == 0; + contentArrays[leadBit] = array = new AtomicReferenceArray<>(CONTENTS_START_SIZE << leadBit); + } + array.lazySet(ofs, value); // no need for a volatile set here; at this point the item is not referenced + // by any node in the trie, and a volatile set will be made to reference it. + return index; + } + + private void setContent(int index, T value) + { + int leadBit = getChunkIdx(index, CONTENTS_START_SHIFT, CONTENTS_START_SIZE); + int ofs = getChunkOffset(index, leadBit, CONTENTS_START_SIZE); + AtomicReferenceArray array = contentArrays[leadBit]; + array.set(ofs, value); + } + + public void discardBuffers() + { + if (bufferType == BufferType.ON_HEAP) + return; // no cleaning needed + + for (UnsafeBuffer b : buffers) + { + if (b != null) + FileUtils.clean(b.byteBuffer()); + } + } + + // Write methods + + // Write visibility model: writes are not volatile, with the exception of the final write before a call returns + // the same value that was present before (e.g. content was updated in-place / existing node got a new child or had + // a child pointer updated); if the whole path including the root node changed, the root itself gets a volatile + // write. + // This final write is the point where any new cells created during the write become visible for readers for the + // first time, and such readers must pass through reading that pointer, which forces a happens-before relationship + // that extends to all values written by this thread before it. + + /** + * Attach a child to the given non-content node. This may be an update for an existing branch, or a new child for + * the node. An update _is_ required (i.e. this is only called when the newChild pointer is not the same as the + * existing value). + */ + private int attachChild(int node, int trans, int newChild) throws SpaceExhaustedException + { + if (isLeaf(node)) + throw new AssertionError("attachChild cannot be used on content nodes."); + + switch (offset(node)) + { + case PREFIX_OFFSET: + throw new AssertionError("attachChild cannot be used on content nodes."); + case SPARSE_OFFSET: + return attachChildToSparse(node, trans, newChild); + case SPLIT_OFFSET: + attachChildToSplit(node, trans, newChild); + return node; + case LAST_POINTER_OFFSET - 1: + // If this is the last character in a Chain block, we can modify the child in-place + if (trans == getByte(node)) + { + putIntVolatile(node + 1, newChild); + return node; + } + // else pass through + default: + return attachChildToChain(node, trans, newChild); + } + } + + /** + * Attach a child to the given split node. This may be an update for an existing branch, or a new child for the node. + */ + private void attachChildToSplit(int node, int trans, int newChild) throws SpaceExhaustedException + { + int midPos = node + SPLIT_POINTER_OFFSET + splitNodeMidIndex(trans) * 4; + int mid = getInt(midPos); + if (isNull(mid)) + { + mid = allocateBlock(); + putIntOrdered(midPos, mid); // ordered write to ensure no uncleaned state is visible to readers + // i.e. if block is reused it may need to be set to all zero. if this is not ordered the writes clearing + // it may execute after this link is created, and readers could see old content. + // Not currently necessary (we don't reuse), but let's avoid the surprise when we start doing so. + } + + int tailPos = mid + splitNodeTailIndex(trans) * 4; + int tail = getInt(tailPos); + if (isNull(tail)) + { + tail = allocateBlock(); + putIntOrdered(tailPos, tail); // as above + } + + int childPos = tail + splitNodeChildIndex(trans) * 4; + putIntVolatile(childPos, newChild); + } + + /** + * Attach a child to the given sparse node. This may be an update for an existing branch, or a new child for the node. + */ + private int attachChildToSparse(int node, int trans, int newChild) throws SpaceExhaustedException + { + int i; + // first check if this is an update and modify in-place if so + for (i = 0; i < SPARSE_CHILD_COUNT; ++i) + { + if (isNull(getInt(node + SPARSE_CHILDREN_OFFSET + i * 4))) + break; + if ((getByte(node + SPARSE_BYTES_OFFSET + i)) == trans) + { + putIntVolatile(node + SPARSE_CHILDREN_OFFSET + i * 4, newChild); + return node; + } + } + + if (i == SPARSE_CHILD_COUNT) + { + // Node is full. Switch to split + int split = createEmptySplitNode(); + for (i = 0; i < SPARSE_CHILD_COUNT; ++i) + { + int t = getByte(node + SPARSE_BYTES_OFFSET + i); + int p = getInt(node + SPARSE_CHILDREN_OFFSET + i * 4); + attachChildToSplitNonVolatile(split, t, p); + } + attachChildToSplitNonVolatile(split, trans, newChild); + return split; + } + + // Add a new transition. They are not kept in order, so append it at the first free position. + putByte(node + SPARSE_BYTES_OFFSET + i, (byte) trans); + + // Update order word. + int order = getShort(node + SPARSE_ORDER_OFFSET) & 0xFFFF; + int newOrder = insertInOrderWord(order, i, trans, node + SPARSE_BYTES_OFFSET); + + // Sparse nodes have two access modes: via the order word, when listing transitions, or directly to characters + // and addresses. + // To support the former, we volatile write to the order word last, and everything is correctly set up. + // The latter does not touch the order word. To support that too, we volatile write the address, as the reader + // can't determine if the position is in use based on the character byte alone (00 is also a valid transition). + // Note that this means that reader must check the transition byte AFTER the address, to ensure they get the + // correct value (see getSparseChild). + + // setting child enables reads to start seeing the new branch + putIntVolatile(node + SPARSE_CHILDREN_OFFSET + i * 4, newChild); + + // some readers will decide whether to check the pointer based on the order word + // write that volatile to make sure they see the new change too + putShortVolatile(node + SPARSE_ORDER_OFFSET, (short) newOrder); + return node; + } + + /** + * Insert the given newIndex in the base-6 encoded order word in the correct position with respect to the ordering. + * + * E.g. if the existing bytes were 20, 50, 30 with order word 120 (decimal 48), then + * - insertOrderWord(120, 3, 5, ptr) must return 1203 (decimal 48*6 + 3) + * - insertOrderWord(120, 3, 25, ptr) must return 1230 (decimal 8*36 + 3*6 + 0) + * - insertOrderWord(120, 3, 35, ptr) must return 1320 (decimal 1*216 + 3*36 + 12) + * - insertOrderWord(120, 3, 55, ptr) must return 3120 (decimal 3*216 + 48) + */ + private int insertInOrderWord(int order, int newIndex, int transitionByte, int bytesPosition) + { + int s = order; + int r = 1; + while (s != 0) + { + int b = getByte(bytesPosition + s % SPARSE_CHILD_COUNT); + if (b > transitionByte) + break; + + assert b < transitionByte; + r *= 6; + s /= 6; + } + // insert i after the ones we have passed (order % r) and before the remaining (s) + return order % r + (s * 6 + newIndex) * r; + } + + /** + * Non-volatile version of attachChildToSplit. Used when the split node is not reachable yet (during the conversion + * from sparse). + */ + private void attachChildToSplitNonVolatile(int node, int trans, int newChild) throws SpaceExhaustedException + { + int midPos = node + SPLIT_POINTER_OFFSET + splitNodeMidIndex(trans) * 4; + int mid = getInt(midPos); + if (isNull(mid)) + { + mid = allocateBlock(); + putInt(midPos, mid); + } + + int tailPos = mid + splitNodeTailIndex(trans) * 4; + int tail = getInt(tailPos); + if (isNull(tail)) + { + tail = allocateBlock(); + putInt(tailPos, tail); + } + + int childPos = tail + splitNodeChildIndex(trans) * 4; + putInt(childPos, newChild); + } + + /** + * Attach a child to the given chain node. This may be an update for an existing branch with different target + * address, or a second child for the node. + * This method always copies the node -- with the exception of updates that change the child of the last node in a + * chain block with matching transition byte (which this method is not used for, see attachChild), modifications to + * chain nodes cannot be done in place, either because we introduce a new transition byte and have to convert from + * the single-transition chain type to sparse, or because we have to remap the child from the implicit node + 1 to + * something else. + */ + private int attachChildToChain(int node, int transitionByte, int newChild) throws SpaceExhaustedException + { + int existingByte = getByte(node); + if (transitionByte == existingByte) + { + // This will only be called if new child is different from old, and the update is not on the final child + // where we can change it in place (see attachChild). We must always create something new. + // If the child is a chain, we can expand it (since it's a different value, its branch must be new and + // nothing can already reside in the rest of the block). + return expandOrCreateChainNode(transitionByte, newChild); + } + + // The new transition is different, so we no longer have only one transition. Change type. + int existingChild = node + 1; + if (offset(existingChild) == LAST_POINTER_OFFSET) + { + existingChild = getInt(existingChild); + } + return createSparseNode(existingByte, existingChild, transitionByte, newChild); + } + + private boolean isExpandableChain(int newChild) + { + int newOffset = offset(newChild); + return newChild > 0 && newChild - 1 > NONE && newOffset > CHAIN_MIN_OFFSET && newOffset <= CHAIN_MAX_OFFSET; + } + + /** + * Create a sparse node with two children. + */ + private int createSparseNode(int byte1, int child1, int byte2, int child2) throws SpaceExhaustedException + { + assert byte1 != byte2; + if (byte1 > byte2) + { + // swap them so the smaller is byte1, i.e. there's always something bigger than child 0 so 0 never is + // at the end of the order + int t = byte1; byte1 = byte2; byte2 = t; + t = child1; child1 = child2; child2 = t; + } + + int node = allocateBlock() + SPARSE_OFFSET; + putByte(node + SPARSE_BYTES_OFFSET + 0, (byte) byte1); + putByte(node + SPARSE_BYTES_OFFSET + 1, (byte) byte2); + putInt(node + SPARSE_CHILDREN_OFFSET + 0 * 4, child1); + putInt(node + SPARSE_CHILDREN_OFFSET + 1 * 4, child2); + putShort(node + SPARSE_ORDER_OFFSET, (short) (1 * 6 + 0)); + // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be + // put in an existing node or the root. That action ends in a happens-before enforcing write. + return node; + } + + /** + * Creates a chain node with the single provided transition (pointing to the provided child). + * Note that to avoid creating inefficient tries with under-utilized chain nodes, this should only be called from + * {@link #expandOrCreateChainNode} and other call-sites should call {@link #expandOrCreateChainNode}. + */ + private int createNewChainNode(int transitionByte, int newChild) throws SpaceExhaustedException + { + int newNode = allocateBlock() + LAST_POINTER_OFFSET - 1; + putByte(newNode, (byte) transitionByte); + putInt(newNode + 1, newChild); + // Note: this does not need a volatile write as it is a new node, returning a new pointer, which needs to be + // put in an existing node or the root. That action ends in a happens-before enforcing write. + return newNode; + } + + /** Like {@link #createNewChainNode}, but if the new child is already a chain node and has room, expand + * it instead of creating a brand new node. */ + private int expandOrCreateChainNode(int transitionByte, int newChild) throws SpaceExhaustedException + { + if (isExpandableChain(newChild)) + { + // attach as a new character in child node + int newNode = newChild - 1; + putByte(newNode, (byte) transitionByte); + return newNode; + } + + return createNewChainNode(transitionByte, newChild); + } + + private int createEmptySplitNode() throws SpaceExhaustedException + { + int pos = allocateBlock(); + return pos + SPLIT_OFFSET; + } + + private int createContentNode(int contentIndex, int child, boolean isSafeChain) throws SpaceExhaustedException + { + assert !isLeaf(child); + if (isNull(child)) + return ~contentIndex; + + int offset = offset(child); + int node; + if (offset == SPLIT_OFFSET || isSafeChain && offset > (PREFIX_FLAGS_OFFSET + PREFIX_OFFSET) && offset <= CHAIN_MAX_OFFSET) + { + // We can do an embedded prefix node + // Note: for chain nodes we have a risk that the node continues beyond the current point, in which case + // creating the embedded node may overwrite information that is still needed by concurrent readers or the + // mutation process itself. + node = (child & -BLOCK_SIZE) | PREFIX_OFFSET; + putByte(node + PREFIX_FLAGS_OFFSET, (byte) offset); + } + else + { + // Full prefix node + node = allocateBlock() + PREFIX_OFFSET; + putByte(node + PREFIX_FLAGS_OFFSET, (byte) 0xFF); + putInt(node + PREFIX_POINTER_OFFSET, child); + } + + putInt(node + PREFIX_CONTENT_OFFSET, contentIndex); + return node; + } + + private int updatePrefixNodeChild(int node, int child) throws SpaceExhaustedException + { + assert offset(node) == PREFIX_OFFSET; + + if (isNull(child)) + return ~getInt(node + PREFIX_CONTENT_OFFSET); + + // We can only update in-place if we have a full prefix node + if (!isEmbeddedPrefixNode(node)) + { + // This attaches the child branch and makes it reachable -- the write must be volatile. + putIntVolatile(node + PREFIX_POINTER_OFFSET, child); + return node; + } + else + { + int contentIndex = getInt(node + PREFIX_CONTENT_OFFSET); + return createContentNode(contentIndex, child, true); + } + } + + private boolean isEmbeddedPrefixNode(int node) + { + return getByte(node + PREFIX_FLAGS_OFFSET) < BLOCK_SIZE; + } + + /** + * Copy the content from an existing node, if it has any, to a newly-prepared update for its child. + * + * @param existingPreContentNode pointer to the existing node before skipping over content nodes, i.e. this is + * either the same as existingPostContentNode or a pointer to a prefix or leaf node + * whose child is existingPostContentNode + * @param existingPostContentNode pointer to the existing node being updated, after any content nodes have been + * skipped and before any modification have been applied; always a non-content node + * @param updatedPostContentNode is the updated node, i.e. the node to which all relevant modifications have been + * applied; if the modifications were applied in-place, this will be the same as + * existingPostContentNode, otherwise a completely different pointer; always a non- + * content node + * @return a node which has the children of updatedPostContentNode combined with the content of + * existingPreContentNode + */ + private int preserveContent(int existingPreContentNode, + int existingPostContentNode, + int updatedPostContentNode) throws SpaceExhaustedException + { + if (existingPreContentNode == existingPostContentNode) + return updatedPostContentNode; // no content to preserve + + if (existingPostContentNode == updatedPostContentNode) + return existingPreContentNode; // child didn't change, no update necessary + + // else we have existing prefix node, and we need to reference a new child + if (isLeaf(existingPreContentNode)) + { + assert isNull(existingPostContentNode); + return createContentNode(~existingPreContentNode, updatedPostContentNode, true); + } + + assert offset(existingPreContentNode) == PREFIX_OFFSET; + return updatePrefixNodeChild(existingPreContentNode, updatedPostContentNode); + } + + final ApplyState applyState = new ApplyState(); + + /** + * State of the walk of the given mutation trie. Passed to mutation nodes in their parentState link. + */ + class ApplyState + { + int[] data = new int[16 * 5]; + int currentLevel = -1; + + void reset() + { + currentLevel = -1; + } + + /** + * Pointer to the existing node before skipping over content nodes, i.e. this is either the same as + * existingPostContentNode or a pointer to a prefix or leaf node whose child is existingPostContentNode. + */ + int existingPreContentNode() + { + return data[currentLevel * 5 + 0]; + } + void setExistingPreContentNode(int value) + { + data[currentLevel * 5 + 0] = value; + } + + /** + * Pointer to the existing node being updated, after any content nodes have been skipped and before any + * modification have been applied. Always a non-content node. + */ + int existingPostContentNode() + { + return data[currentLevel * 5 + 1]; + } + void setExistingPostContentNode(int value) + { + data[currentLevel * 5 + 1] = value; + } + + /** + * The updated node, i.e. the node to which the relevant modifications are being applied. This will change as + * children are processed and attached to the node. After all children have been processed, this will contain + * the fully updated node (i.e. the union of existingPostContentNode and mutationNode) without any content, + * which will be processed separately and, if necessary, attached ahead of this. If the modifications were + * applied in-place, this will be the same as existingPostContentNode, otherwise a completely different + * pointer. Always a non-content node. + */ + int updatedPostContentNode() + { + return data[currentLevel * 5 + 2]; + } + + void setUpdatedPostContentNode(int value) + { + data[currentLevel * 5 + 2] = value; + } + + int transition() + { + return data[currentLevel * 5 + 3]; + } + void setTransition(int transition) + { + data[currentLevel * 5 + 3] = transition; + } + int contentIndex() + { + return data[currentLevel * 5 + 4]; + } + void setContentIndex(int value) + { + data[currentLevel * 5 + 4] = value; + } + + void descend(int transition, U mutationContent, final UpsertTransformer transformer) + { + int existingPreContentNode; + if (currentLevel < 0) + existingPreContentNode = root; + else + { + setTransition(transition); + existingPreContentNode = isNull(existingPostContentNode()) + ? NONE + : getChild(existingPostContentNode(), transition); + } + + ++currentLevel; + if (currentLevel * 5 >= data.length) + data = Arrays.copyOf(data, currentLevel * 5 * 2); + setExistingPreContentNode(existingPreContentNode); + + int existingContentIndex = -1; + int existingPostContentNode; + if (isLeaf(existingPreContentNode)) + { + existingContentIndex = ~existingPreContentNode; + existingPostContentNode = NONE; + } + else if (offset(existingPreContentNode) == PREFIX_OFFSET) + { + existingContentIndex = getInt(existingPreContentNode + PREFIX_CONTENT_OFFSET); + existingPostContentNode = followContentTransition(existingPreContentNode); + } + else + existingPostContentNode = existingPreContentNode; + setExistingPostContentNode(existingPostContentNode); + setUpdatedPostContentNode(existingPostContentNode); + + int contentIndex = updateContentIndex(mutationContent, existingContentIndex, transformer); + setContentIndex(contentIndex); + } + + private int updateContentIndex(U mutationContent, int existingContentIndex, final UpsertTransformer transformer) + { + if (mutationContent != null) + { + if (existingContentIndex != -1) + { + final T existingContent = getContent(existingContentIndex); + T combinedContent = transformer.apply(existingContent, mutationContent); + setContent(existingContentIndex, combinedContent); + if (combinedContent != null) + return existingContentIndex; + else + return -1; + } + else + { + T combinedContent = transformer.apply(null, mutationContent); + if (combinedContent != null) + return addContent(combinedContent); + else + return -1; + } + } + else + return existingContentIndex; + } + + private void attachChild(int transition, int ourChild) throws SpaceExhaustedException + { + int updatedPostContentNode = updatedPostContentNode(); + if (isNull(updatedPostContentNode)) + setUpdatedPostContentNode(expandOrCreateChainNode(transition, ourChild)); + else + setUpdatedPostContentNode(MemtableTrie.this.attachChild(updatedPostContentNode, + transition, + ourChild)); + } + + private int applyContent() throws SpaceExhaustedException + { + int contentIndex = contentIndex(); + int updatedPostContentNode = updatedPostContentNode(); + if (contentIndex == -1) + return updatedPostContentNode; + + if (isNull(updatedPostContentNode)) + return ~contentIndex; + + int existingPreContentNode = existingPreContentNode(); + int existingPostContentNode = existingPostContentNode(); + + // We can't update in-place if there was no preexisting prefix, or if the prefix was embedded and the target + // node must change. + if (existingPreContentNode == existingPostContentNode || + isNull(existingPostContentNode) || + isEmbeddedPrefixNode(existingPreContentNode) && updatedPostContentNode != existingPostContentNode) + return createContentNode(contentIndex, updatedPostContentNode, isNull(existingPostContentNode)); + + // Otherwise modify in place + if (updatedPostContentNode != existingPostContentNode) // to use volatile write but also ensure we don't corrupt embedded nodes + putIntVolatile(existingPreContentNode + PREFIX_POINTER_OFFSET, updatedPostContentNode); + assert contentIndex == getInt(existingPreContentNode + PREFIX_CONTENT_OFFSET); + return existingPreContentNode; + } + + // true if still have work to do, false if completed + private boolean attachAndMoveToParentState() throws SpaceExhaustedException + { + int updatedPreContentNode = applyContent(); + int existingPreContentNode = existingPreContentNode(); + --currentLevel; + if (currentLevel == -1) + { + assert root == existingPreContentNode; + if (updatedPreContentNode != existingPreContentNode) + { + // Only write to root if they are different (value doesn't change, but + // we don't want to invalidate the value in other cores' caches unnecessarily). + root = updatedPreContentNode; + } + return false; + } + if (updatedPreContentNode != existingPreContentNode) + attachChild(transition(), updatedPreContentNode); + return true; + } + } + + /** + * Somewhat similar to {@link MergeResolver}, this encapsulates logic to be applied whenever new content is being + * upserted into a {@link MemtableTrie}. Unlike {@link MergeResolver}, {@link UpsertTransformer} will be applied no + * matter if there's pre-existing content for that trie key/path or not. + * + * @param The content type for this {@link MemtableTrie}. + * @param The type of the new content being applied to this {@link MemtableTrie}. + */ + public interface UpsertTransformer + { + /** + * Called when there's content in the updating trie. + * + * @param existing Existing content for this key, or null if there isn't any. + * @param update The update, always non-null. + * @return The combined value to use. + */ + T apply(T existing, U update); + } + + /** + * Modify this trie to apply the mutation given in the form of a trie. Any content in the mutation will be resolved + * with the given function before being placed in this trie (even if there's no pre-existing content in this trie). + * @param mutation the mutation to be applied, given in the form of a trie. Note that its content can be of type + * different than the element type for this memtable trie. + * @param transformer a function applied to the potentially pre-existing value for the given key, and the new + * value. Applied even if there's no pre-existing value in the memtable trie. + */ + public void apply(Trie mutation, final UpsertTransformer transformer) throws SpaceExhaustedException + { + Cursor mutationCursor = mutation.cursor(); + if (mutationCursor.level() == -1) + return; + assert mutationCursor.level() == 0; + ApplyState state = applyState; + state.reset(); + state.descend(-1, mutationCursor.content(), transformer); + assert state.currentLevel == 0; + + while (true) + { + int level = mutationCursor.advance(); + while (state.currentLevel >= level) + { + // There are no more children. Ascend to the parent state to continue walk. + if (!state.attachAndMoveToParentState()) + { + assert level == -1; + return; + } + } + + // We have a transition, get child to descend into + state.descend(mutationCursor.incomingTransition(), mutationCursor.content(), transformer); + assert state.currentLevel == level; + } + } + + /** + * Map-like put method, using the apply machinery above which cannot run into stack overflow. When the correct + * position in the trie has been reached, the value will be resolved with the given function before being placed in + * the trie (even if there's no pre-existing content in this trie). + * @param key the trie path/key for the given value. + * @param value the value being put in the memtable trie. Note that it can be of type different than the element + * type for this memtable trie. It's up to the {@code transformer} to return the final value that will stay in + * the memtable trie. + * @param transformer a function applied to the potentially pre-existing value for the given key, and the new + * value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied + * even if there's no pre-existing value in the memtable trie. + */ + public void putSingleton(ByteComparable key, + R value, + UpsertTransformer transformer) throws SpaceExhaustedException + { + apply(Trie.singleton(key, value), transformer); + } + + /** + * A version of putSingleton which uses recursive put if the last argument is true. + */ + public void putSingleton(ByteComparable key, + R value, + UpsertTransformer transformer, + boolean useRecursive) throws SpaceExhaustedException + { + if (useRecursive) + putRecursive(key, value, transformer); + else + putSingleton(key, value, transformer); + } + + /** + * Map-like put method, using a fast recursive implementation through the key bytes. May run into stack overflow if + * the trie becomes too deep. When the correct position in the trie has been reached, the value will be resolved + * with the given function before being placed in the trie (even if there's no pre-existing content in this trie). + * @param key the trie path/key for the given value. + * @param value the value being put in the memtable trie. Note that it can be of type different than the element + * type for this memtable trie. It's up to the {@code transformer} to return the final value that will stay in + * the memtable trie. + * @param transformer a function applied to the potentially pre-existing value for the given key, and the new + * value (of a potentially different type), returning the final value that will stay in the memtable trie. Applied + * even if there's no pre-existing value in the memtable trie. + */ + public void putRecursive(ByteComparable key, R value, final UpsertTransformer transformer) throws SpaceExhaustedException + { + int newRoot = putRecursive(root, key.asComparableBytes(BYTE_COMPARABLE_VERSION), 0, value, transformer); + if (newRoot != root) + root = newRoot; + } + + @DontInline + private int putRecursive(int node, ByteSource key, int depth, R value, final UpsertTransformer transformer) throws SpaceExhaustedException + { + int transition = key.next(); + if (transition == ByteSource.END_OF_STREAM) + return applyContent(node, value, transformer); + + int child = NONE; + if (!isNull(node)) + child = getChild(node, transition); + + int newChild = putRecursive(child, key, depth + 1, value, transformer); + if (newChild == child) + return node; + + int skippedContent = followContentTransition(node); + int attachedChild = !isNull(skippedContent) + ? attachChild(skippedContent, transition, newChild) // Single path, no copying required + : expandOrCreateChainNode(transition, newChild); + + return preserveContent(node, skippedContent, attachedChild); + } + + private int applyContent(int node, R value, UpsertTransformer transformer) throws SpaceExhaustedException + { + if (isNull(node)) + return ~addContent(transformer.apply(null, value)); + + if (isLeaf(node)) + { + int contentIndex = ~node; + setContent(contentIndex, transformer.apply(getContent(contentIndex), value)); + return node; + } + + if (offset(node) == PREFIX_OFFSET) + { + int contentIndex = getInt(node + PREFIX_CONTENT_OFFSET); + setContent(contentIndex, transformer.apply(getContent(contentIndex), value)); + return node; + } + else + return createContentNode(addContent(transformer.apply(null, value)), node, false); + } + + /** + * Returns true if the allocation threshold has been reached. To be called by the the writing thread (ideally, just + * after the write completes). When this returns true, the user should switch to a new trie as soon as feasible. + * + * The trie expects up to 10% growth above this threshold. Any growth beyond that may be done inefficiently, and + * the trie will fail altogether when the size grows beyond 2G - 256 bytes. + */ + public boolean reachedAllocatedSizeThreshold() + { + return allocatedPos >= ALLOCATED_SIZE_THRESHOLD; + } + + /** + * For tests only! Advance the allocation pointer (and allocate space) by this much to test behaviour close to + * full. + */ + @VisibleForTesting + int advanceAllocatedPos(int wantedPos) throws SpaceExhaustedException + { + while (allocatedPos < wantedPos) + allocateBlock(); + return allocatedPos; + } + + /** Returns the off heap size of the memtable trie itself, not counting any space taken by referenced content. */ + public long sizeOffHeap() + { + return bufferType == BufferType.ON_HEAP ? 0 : allocatedPos; + } + + /** Returns the on heap size of the memtable trie itself, not counting any space taken by referenced content. */ + public long sizeOnHeap() + { + return contentCount * MemoryLayoutSpecification.SPEC.getReferenceSize() + + (bufferType == BufferType.ON_HEAP ? allocatedPos + EMPTY_SIZE_ON_HEAP : EMPTY_SIZE_OFF_HEAP); + } + + @Override + public Iterable valuesUnordered() + { + return () -> new Iterator() + { + int idx = 0; + + public boolean hasNext() + { + return idx < contentCount; + } + + public T next() + { + if (!hasNext()) + throw new NoSuchElementException(); + + return getContent(idx++); + } + }; + } + + public int valuesCount() + { + return contentCount; + } +} diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md new file mode 100644 index 000000000000..1f0cfed6cd41 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md @@ -0,0 +1,754 @@ + + +# MemtableTrie Design + +The `MemtableTrie` is one of the main components of the trie infrastructure, a mutable in-memory trie built for fast +modification and reads executing concurrently with writes from a single mutator thread. + +The main features of its implementation is: +- support for writes from a single mutator thread concurrent with multiple readers +- full support of the Trie interface +- uses nodes of several different types for efficiency +- supports content on any node, including intermediate (prefix) +- upper limit for the trie size + + +## Memory layout + +One of the main design drivers of the memtable trie is the desire to avoid on-heap storage and Java object management. +The trie thus implements its own memory management for the structure of the trie (content is, at this time, still given +as Java objects in a content array). The structure resides in one `UnsafeBuffer` (which can be on or off heap as +desired) and is broken up in 32-byte "cells" (also called "blocks" in the code), which are the unit of allocation, +update and reuse. + +Like all tries, `MemtableTrie` is built from nodes and has a root pointer. The nodes reside in cells, but there is no +1:1 correspondence between nodes and cells - some node types pack multiple in one cell, while other types require +multiple cells. + +### Pointers and node types + +A "pointer" is an integer that points to a node in the trie buffer. A pointer specifies the location of the node +(its starting cell), but also defines the type of node in its 5 lowest-order bits (i.e. the offset within the cell). +If a pointer has a negative value, it refers to a value in the content array, and implies a leaf node with the specified +content. Additionally, the special pointer value `NONE` (0) is used to specify "no child". We use 32-bit integers as +pointers, therefore the size of the trie structure is limited to a little less than 2GB. + +For example, the pointer `0x0109E` specifies a node residing in the cell at bytes `0x01080`-`0x0109F` in the buffer +(specified by the pointers' 27 leading bits), where the node type is `Sparse` (specified by `0x1E` in the last 5 bits). + +The pointer `0xFFFFFFF0` specifies a leaf node (being negative), where the content's index is `0xF` (obtained by +negating all bits of the pointer). + +To save space and reduce pointer chasing, we use several different types of nodes that address different common patterns +in a trie. It is common for a trie to have one or a couple of top levels which have many children, and where it is +important to make decisions with as few if-then-else branches as possible (served by the `Split` type), another one or +two levels of nodes with a small number of children, where it is most important to save space as the number of these +nodes is high (served by the `Sparse` type), and a lot of sequences of single-child nodes containing the trailing bytes +of the key or of some common key prefix (served by the `Chain` type). Most of the payload/content of the trie resides +at the leaves, where it makes sense to avoid taking any space for a node (the `Leaf` type), but we must also allow the +possibility for values to be present in intermediate nodes — because this is rare, we support it with a special +`Prefix` type instead of reserving a space for payload in all other node types. + +The Split-Sparse-Chain-Leaf/Prefix pattern may repeat several times. For example, we could have these four layers for +the partition key with some metadata associated with the partition, then for the first component of the clustering key, +then for the second component etc. + +The sections below specify the layout of each supported node type. + +#### Leaf nodes + +Leaf nodes do not have a corresponding cell in the buffer. Instead, they reference a value (i.e. a POJO in the +`MemtableTrie`'s content type) in the content array. The index of the value is specified by `~pointer` (unlike `-x`, +`~x` allows one to also encode 0 in a negative number). + +Leaf nodes have no children, and return the specified value for `content()`. + +Example: -1 is a leaf cell with content `contentArray[0]`. + +#### `Chain` nodes - single path, multiple transitions in one cell + +Chain nodes are one-child nodes. Multiple chain nodes, forming a chain of transitions to one target, can reside in a +single cell. Chain nodes are identified by the lowest 5 bits of a pointer being between `0x00` and `0x1B`. In addition +to the the type of node, in this case the bits also define the length of the chain — the difference between +`0x1C` and the pointer offset specifies the number of characters in the chain. + +The simplest chain node has one transition leading to one child and is laid out like this: + +offset|content|example +---|---|--- +00 - 1A|unused| +1B |character|41 A +1C - 1F|child pointer|FFFFFFFF + +where the pointer points to the `1B` line in the cell. + +Example: The cell `xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxx41 FFFFFFFF` at bytes `0x120`-`0x13F` and +pointer `0x13B` point to a node with one child with transition `0x41` `A` to a leaf node with content `contentArray[0]`. + +Another chain cell, which points to this one, can be added in the same cell by placing a character at offset `1A`. This +new node is effectively laid out as + +offset|content|example +---|---|--- +00 - 19|unused| +1A |character|48 H +1B - 1F|unused| + +where the pointer points to line `1A`. This node has one transition, and the child pointer is implicit as the node's +pointer plus one. + +This can continue until all the bytes in the "unused" area are filled. + +Example: The cell `xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xxxxxxxx xx434841 FFFFFFFF` at bytes `0x120`-`0x13F` and +pointer `0x139` point to a node with one child with transition `0x43` `C` to a node with one child with transition +`0x48` `H` to a node with one child with transition `0x41` `A` to a leaf node with content `contentArray[0]`. + +offset|content|example +---|---|--- +00 - 18|unused| +19 |character|43 C +1A |character|48 H +1B |character|41 A +1C - 1F|child pointer|FFFFFFFF + + +In this example `0x13A` and `0x13B` are also valid pointers to the respective chains and could be referenced from other +nodes (an example will be given below). In any case, the byte pointed directly by the node pointer contains the +transition byte. The child pointer is either `pointer + 1` (if the lowest 5 pointer bits are less than `0x1B`), or the +integer stored at `pointer + 1` (if the pointer's last 5 bits are `0x1B`). + +![graph](MemtableTrie.md.g1.svg) + +Note: offset `0x00` also specifies a chain node, but the pointer 0 is a special case and care must be taken to ensure no +28-byte chain node is placed in the cell at bytes `0x00`-`0x1F`. + +#### `Sparse` nodes - between 2 and 6 children in one cell + +Sparse nodes are used when a node has at least two children, and all pointers and transition characters can fit in one +cell, which limits the maximum number of children to 6. Their layout is: + +offset|content| +---|---| +00 - 03|child pointer 0| +04 - 07|child pointer 1| +08 - 0B|child pointer 2| +0C - 0F|child pointer 3| +10 - 13|child pointer 4| +14 - 17|child pointer 5| +18 |character 0| +19 |character 1| +1A |character 2| +1B |character 3| +1C |character 4| +1D |character 5| +1E - 1F|order word| + +where the pointer points to the line `1E` (i.e. the type identifier for a sparse node is `0x1E`). + +It is important to note that the pointers and characters are not in order. This is done so that an update to a sparse +node where a new child is inserted can be done while the previous state of the node is still valid and readable for +any concurrent readers. Instead, new children are appended, and the order is maintained in the "order word". This word +is a number whose digits specify the order of the children's transition characters (where higher-order digits specify +bigger characters) encoded, to be able to fit into a 16-bit word, in base 6. Its number of digits also specifies the +number of children of the node. + +To explain this better, we will give an example of the evolution of a sparse node. Suppose we had the `0x139` node from +the previous section, and some update needs to attach a second child to that, e.g. with the character `A` and child +`0x238`. + +![graph](MemtableTrie.md.g2.svg) + +To do this, the mutating thread will have to convert the chain node into a sparse by allocating a new cell +(e.g. `0x240`-`0x25F`) and filling in the sparse node `00000238 0000013A 00000000 00000000 00000000 00000000 41430000 +00000006` with pointer `0x25E`: + +offset|content|example +---|---|--- +00 - 03|child pointer 0| 00000238 +04 - 07|child pointer 1| 0000013A +08 - 17|unused| +18 |character 0| 41 A +19 |character 1| 43 C +1A - 1D|unused| +1E - 1F|order word, always 10| 0006 = 10 (base 6) + +This is the smallest kind of sparse node, with just two children. Two-children sparse nodes always +put their two children in order (we can do this as this does not happen in response to an addition of a new child to +an existing sparse node, but this is constructed directly) and thus their order word is always 10 (if they were +not in order, the order word would have to be 01, which would be misinterpreted as the single-digit 1). + +This node has two (the number of digits in the order word) children. The first child is at the position specified by the +least significant digit of the order word, 0. The second child is specified by the second least significant digit, 1. + +Suppose we then need to add a new child, using character `0x35` `5` and child `0x33B`. The node will change to `00000238 +0000013A 0000033B 00000000 00000000 00000000 41433500 00000026` and the pointer to it stays the same. + +offset|content|example +---|---|--- +00 - 03|child pointer 0| 00000238 +04 - 07|child pointer 1| 0000013A +08 - 0B|child pointer 2| 0000033B +0C - 17|unused| +18 |character 0| 41 A +19 |character 1| 43 C +1A |character 2| 35 5 +1B - 1D|unused| +1E - 1F|order word| 0026 = 102 (base 6) + +This node has three (the number of digits in the order word) children. The first child is at the position specified by +the least significant digit of the order word, 2. The second child is specified by the second least significant digit, +0, and the last child is specified by the leading digit, 1. + +Note that because of the ordering of the two children in the smallest sparse node, the digit 0 is always preceded by a +more-significant 1 in the order word in base 6. Therefore the leading digit of the order word can never be 0 and thus we +cannot miscount the number of children. + +The addition of children can continue until we have 6, for example `00000238 0000013A 0000033B 0000035C 0000037A +0000041B 41433542 50338129` (pointer `0x25E`) for + +offset|content|example +---|---|--- +00 - 03|child pointer 0| 00000238 +04 - 07|child pointer 1| 0000013A +08 - 0B|child pointer 2| 0000033B +0C - 0F|child pointer 3| 0000035C +10 - 13|child pointer 4| 0000037A +14 - 17|child pointer 5| 0000041B +18 |character 0| 41 A +19 |character 1| 43 C +1A |character 2| 35 5 +1B |character 3| 42 B +1C |character 4| 50 P +1D |character 5| 33 3 +1E - 1F|order word| 8129 = 413025 (base 6) + +Beyond 6 children, a node needs to be converted to split. + +#### `Split` nodes - up to 256 children in multiple cells + +Split nodes are used to handle the nodes with a large number of children. We can only allocate cells of 32 bytes, thus +we have to distribute the child transitions among cells in some way that is efficient for reading and updating. The +method we chose is to construct a "mini-trie" with 2-3-3 bit transitions. + +A split node is identified by the `0x1C` offset. The starting cell of a split node has this layout: + +offset|content| +---|---| +00 - 0F|unused| +10 - 13|mid-cell for leading 00| +14 - 17|mid-cell for leading 01| +18 - 1B|mid-cell for leading 10| +1C - 1F|mid-cell for leading 11| + +(pointers to this node point to the `1C` line) and where each mid-cell contains: + +offset|content| +---|---| +00 - 03|end-cell for middle 000| +04 - 07|end-cell for middle 001| +08 - 0B|end-cell for middle 010| +0C - 0F|end-cell for middle 011| +10 - 13|end-cell for middle 100| +14 - 17|end-cell for middle 101| +18 - 1B|end-cell for middle 110| +1C - 1F|end-cell for middle 111| + +and end-cell: + +offset|content| +---|---| +00 - 03|pointer to child for ending 000| +04 - 07|pointer to child for ending 001| +08 - 0B|pointer to child for ending 010| +0C - 0F|pointer to child for ending 011| +10 - 13|pointer to child for ending 100| +14 - 17|pointer to child for ending 101| +18 - 1B|pointer to child for ending 110| +1C - 1F|pointer to child for ending 111| + +In any of the cell or pointer positions we can have `NONE`, meaning that such a child (or block of children) does not +exist. At minimum, a split node occupies 3 cells (one leading, one mid and one end), and at maximum — +`1 + 4 + 4*8 = 37` cells i.e. `1184` bytes. If we could allocate contiguous arrays, a full split node would use `1024` +bytes, thus this splitting can add ~15% overhead. However, real data often has additional structure that this can make +use of to avoid creating some of the blocks, e.g. if the trie encodes US-ASCII or UTF-encoded strings where some +character ranges are not allowed at all, and others are prevalent. Another benefit is that to change a transition while +preserving the previous state of the node for concurrent readers we have to only copy three blocks and not the entire +range of children (applications of this will be given later). + +As an example, suppose we need to add a `0x51` `Q` transition to `0x455` to the 6-children sparse node from the previous +section. This will generate the following structure: + +Leading cell (e.g. `0x500`-`0x51F` with pointer `0x51C`) + +offset|content|example +---|---|--- +00 - 0F|unused| +10 - 13|mid-cell for leading 00|00000520 +14 - 17|mid-cell for leading 01|00000560 +18 - 1B|mid-cell for leading 10|00000000 NONE +1C - 1F|mid-cell for leading 11|00000000 NONE + +Mid cell `00` at `0x520`-`0x53F`: + +offset|content|example +---|---|--- +00 - 03|end-cell for middle 000|00000000 NONE +04 - 07|end-cell for middle 001|00000000 NONE +08 - 0B|end-cell for middle 010|00000000 NONE +0C - 0F|end-cell for middle 011|00000000 NONE +10 - 13|end-cell for middle 100|00000000 NONE +14 - 17|end-cell for middle 101|00000000 NONE +18 - 1B|end-cell for middle 110|00000540 +1C - 1F|end-cell for middle 111|00000000 NONE + +End cell `00 110` at `0x540`-`0x55F`: + +offset|content|example +---|---|--- +00 - 03|pointer to child for ending 000|00000000 NONE +04 - 07|pointer to child for ending 001|00000000 NONE +08 - 0B|pointer to child for ending 010|00000000 NONE +0C - 0F|pointer to child for ending 011|0000041B +10 - 13|pointer to child for ending 100|00000000 NONE +14 - 17|pointer to child for ending 101|0000033B +18 - 1B|pointer to child for ending 110|00000000 NONE +1C - 1F|pointer to child for ending 111|00000000 NONE + +Mid cell `01` at `0x560`-`0x57F`: + +offset|content|example +---|---|--- +00 - 03|end-cell for middle 000|00000580 +04 - 07|end-cell for middle 001|00000000 NONE +08 - 0B|end-cell for middle 010|000005A0 +0C - 0F|end-cell for middle 011|00000000 NONE +10 - 13|end-cell for middle 100|00000000 NONE +14 - 17|end-cell for middle 101|00000000 NONE +18 - 1B|end-cell for middle 110|00000000 NONE +1C - 1F|end-cell for middle 111|00000000 NONE + +End cell `01 000` at `0x580`-`0x59F`: + +offset|content|example +---|---|--- +00 - 03|pointer to child for ending 000|00000000 NONE +04 - 07|pointer to child for ending 001|00000238 +08 - 0B|pointer to child for ending 010|0000035C +0C - 0F|pointer to child for ending 011|0000013A +10 - 13|pointer to child for ending 100|00000000 NONE +14 - 17|pointer to child for ending 101|00000000 NONE +18 - 1B|pointer to child for ending 110|00000000 NONE +1C - 1F|pointer to child for ending 111|00000000 NONE + +End cell `01 010` at `0x5A0`-`0x5BF`: + +offset|content|example +---|---|--- +00 - 03|pointer to child for ending 000|0000037A +04 - 07|pointer to child for ending 001|00000455 +08 - 0B|pointer to child for ending 010|00000000 NONE +0C - 0F|pointer to child for ending 011|00000000 NONE +10 - 13|pointer to child for ending 100|00000000 NONE +14 - 17|pointer to child for ending 101|00000000 NONE +18 - 1B|pointer to child for ending 110|00000000 NONE +1C - 1F|pointer to child for ending 111|00000000 NONE + +To find a child in this structure, we follow the transitions along the bits of the mini-trie. For example, for `0x42` +`B` = `0b01000010` we start at `0x51C`, take the `01` pointer to `0x560`, then the `000` pointer to `0x580` and finally +the `010` index to retrieve the node pointer `0x35C`. Note that the intermediate cells (dashed in the diagram) are not +reachable with pointers, they only make sense as substructure of the split node. + +![graph](MemtableTrie.md.g3.svg) + +#### Content `Prefix` + +Prefix nodes are not nodes in themselves, but they add information to the node they lead to. Specifically, they +encode an index in the content array, and a pointer to the node to which this content is attached. In anything other +than the content, they are equivalent to the linked node — i.e. a prefix node pointer has the same children as +the node it links to (another way to see this is as a content-carrying node is one that has an _ε_ transition to the +linked node and no other features except added content). We do not allow more than one prefix to a node (i.e. prefix +can't point to another prefix), and the child of a prefix node cannot be a leaf. + +There are two types of prefixes: +- standalone, which has a full 32-bit pointer to the linked node +- embedded, which occupies unused space in `Chain` or `Split` nodes and specifies the 5-bit offset within the same cell +of the linked node + +Standalone prefixes have this layout: + +offset|content|example +---|---|--- +00 - 03|content index|00000001 +04|standalone flag, 0xFF|FF +05 - 1B|unused| +1C - 1F|linked node pointer|0000025E + +and pointer offset `0x1F`. The sample values above will be the ones used to link a prefix node to our `Sparse` +example, where a prefix cannot be embedded as all the bytes of the cell are in use. + +If we want to attach the same prefix to the `Split` example, we will place this + +offset|content|example +---|---|--- +00 - 03|content index|00000001 +04|embedded offset within cell|1C +05 - 1F|unused| + +_inside_ the leading split cell, with pointer `0x1F`. Since this is an embedded node, the augmented one resides within +the same cell, and thus we need only 5 bits to encode the pointer (the other 27 are the same as the prefix's). +The combined content of the cell at `0x500-0x51F` will then be `00000001 1C000000 00000000 00000000 00000520 00000560 +00000000 00000000`: + +offset|content|example +---|---|--- +00 - 03|content index|00000001 +04|embedded offset within cell|1C +05 - 0F|unused| +10 - 13|mid-cell for leading 00|00000520 +14 - 17|mid-cell for leading 01|00000560 +18 - 1B|mid-cell for leading 10|00000000 NONE +1C - 1F|mid-cell for leading 11|00000000 NONE + +Both `0x51C` and `0x51F` are valid pointers in this cell. The former refers to the plain split node, the latter to its +content-augmented version. The only difference between the two is the result of a call to `Node.content()`. + +![graph](MemtableTrie.md.g4.svg) + + +## Reading a trie + +`MemtableTrie` is mainly meant to be used as an implementation of `Trie`. As such, the main method of retrieval of +information is via some selection (i.e. intersection) of a subtrie followed by a walk over the content in this +subtrie. Straightforward methods for direct retrieval of data by key are also provided, but they are mainly for testing. + +The `Trie` interface relies on nodes keeping track of the state of iteration, so that it can be continued over result +consumption or pauses to retrieve information asynchronously. `MemtableTrie` supports this by providing `Trie.Node` +implementations (residing in `MemtableReadTrie.xxxNode`) with several special features to aid quick walks over the +trie's content: + +- Like all `Trie.Node` descendants, the nodes are stateful and keep track of the parent chain, as well as the current + iteration position. +- `Chain` nodes, which always have a single descendant, implement `getUniqueDescendant` so that walks can jump + straight to the chain's child instead of walking it one character at a time; this also applies on backtracking + — the walk will skip over the intermediate nodes and go directly to the chain's parent. +- `Chain` and `Sparse` nodes return `Remaining.ONE` to iteration requests when the returned item is the last. This + helps with backtracking as it lets the walk know this node does not need to be visited on the backtracking path, + which can jump straight to the parent. + +As an example, suppose we want to walk the following trie: + +![graph](MemtableTrie.md.w1.svg) + +The classic walk descends (blue) on every character and backtracks (pink) to the parent, resulting in the following + walk: + +![graph](MemtableTrie.md.w2.svg) + +Making use of `getUniqueDescendant` skips the intermediate transitions in `Chain` nodes, and also avoids the +backtracking to the start of the chain (as there are no further transitions to examine there), resulting in: + +![graph](MemtableTrie.md.w3.svg) + +Finally, taking advantage of the `Remaining.ONE` returned by the `Sparse` node after the last child has been listed +lets the backtracking avoid returning to that node, simplifying the walk to: + +![graph](MemtableTrie.md.w4.svg) + +In addition to making the walk simpler, shortening the backtracking paths means a smaller walk state representation, +which is quite helpful in keeping the garbage collection cost down. +Technically, this is achieved by getting a child not with the current node in `parentLink`, but directly using +the node's own `parentLink` (see `TrieValuesIterator.getChild` and `TrieIteratorWithKey.getChild`). + +## Mutation + +Mutation of `MemtableTrie` must be done by one thread only (for performance reasons we don't enforce it, user must +make sure that's the case), but writes may be concurrent with multiple reads over the data that is being mutated. The +trie is built to support this by making sure that any modification of a node is safe for any reader that is operating +concurrently. + +The main method for mutating a `MemtableTrie` is `apply`, which merges the structure of another `Trie` in. +`MemtableTrie` also provides simpler recursive method of modification, `putRecursive`, which creates a single +`key -> value` mapping in the trie. We will describe the mutation process starting with a `putRecursive` example. + +### Adding a new key -> value mapping using `putRecursive` + +Suppose we want to insert the value `traverse` into the trie described in the previous paragraph. The recursive +insertion process walks the trie to find corresponding existing nodes for the ones in the path to be inserted. +When it has to leave the existing trie, because it has no entries for the path, the process continues using `NONE` as +the trie node. + +![graph](MemtableTrie.md.m1.svg) + +When it reaches the end of the path, it needs to attach the value. We don't support content in intermediate nodes, so +we expect the matching trie node to either be `NONE` or a leaf node. Here it's `NONE`, so we create a item in the +content array, `contentArray[3]`, put the value in it, and thus form the leaf node `~3` (`0xFFFFFFFC`). The recursive +process returns this to the previous step. + +The previous step must attach a child with the transition `e` to the node `NONE`. Since this is a new node, we do this +by creating a new `Chain` node at address `0x0BB` mapping `e` to `~3` and return that. For the node above, we again +need to attach a child to `NONE`, but this time the child is a `Chain` node, so we can do this by expanding it, i.e. +writing the new character at the address just before the child pointer, and returning that address (note that the +child chain node is newly created, so we can't be overwriting any existing data there). We can do this several more +times. + +![graph](MemtableTrie.md.m2.svg) + +(Light blue specifies the descent path, pink +the values returned, blue stands for newly-created nodes and links, and +light gray for obsoleted nodes and links.) + +In the next step we must attach the child `0x0B8` with transition `v` to the existing `Chain` node `0x018`. This is a +different transition from the one that node already has, so the change cannot be accommodated by a node of type `Chain`, +thus we need to copy this into a new `Sparse` node `0x0DE` with two children, the existing `c -> 0x019` and the new +`v -> 0x0B8` and return `0x0DE` to the parent step. + +The parent step must then change its existing pointer for the character `a` from `0x018` to `0x0DE` which it can do in +place by writing the new value in its pointer cell for `a`. This is the attachment point for the newly created +substructure, i.e. before this, the new nodes were not reachable, and now become reachable; before this, the node +`0x018 ` was reachable, and now becomes unreachable. The attachment is done by a volatile write, to enforce a happens +-before relationship that makes sure that all the new substructure (all written by this thread) is fully readable by all +readers who pass through the new pointer (which is the only way they can reach it). The same happens-before also ensures +that any new readers cannot reach the obsoleted nodes (there may be existing reader threads that are already in them). + +It can then return its address `0x07E` unchanged up, and no changes need to be done in any of the remaining steps. The +process finishes in a new value for `root`, which in this case remains unchanged. + +![graph](MemtableTrie.md.m3.svg) + +The process created a few new nodes (in blue), and made one obsolete (in grey). What can concurrent readers see depends +on where they are at the time the attachment point write is done. Forward traversals, if they are in the path below +`0x07E`, will continue working with the obsoleted data and will not see any of the new changes. If they are above +`0x07E`, they will see the updated content. If they are _at_ the `0x07E` node, they may see either, depending on the +time they read the pointer for `a`. Reverse traversals that happen to be in the region to the right of the new nodes +_will_ see the updated content, as they will read the pointer after it has been updated. + +In any case, the obsolete paths remain correct and usable for any thread that has already reached them, and the new +paths are correct and usable from the moment they become reachable. + +Note that if we perform multiple mutations in sequence, and a reader happens to be stalled between them (in iteration +order), such reader may see only the mutation that is ahead of it _in iteration order_, which is not necessarily the +mutation that happened first. For the example above, if we also inserted `trespass`, a reader thread that was paused +at `0x018` in a forward traversal and wakes up after both insertions have completed will see `trespass`, but _will not_ +see `traverse` even though it was inserted earlier. This inconsistency is often undesirable. + +### In-place modifications + +When the backtracking process returns with a new mapping, there are several cases when we can apply a change in place +(creating an attachment point for the new path). We will explain these in detail, as it is important to understand what +exactly happens from concurrent readers' point of view in all of them. + +Note that if a modification cannot be done in place, we copy the content to a new node. The copied node is always +unreachable and there will always be an attachment point that makes it reachable somewhere in the parent chain. + +#### Changing the child pointer of the last `Chain` node in a chain + +This happens when the existing transition matches the transition of the new character, but the pointer is different, +and only applies to `Chain` nodes whose offset is `0x1B`. In this case the child pointer is written at offset `0x1C`, +and we can put in the new value by performing a volatile write. + +For example, updating `N -> 0x39C` is accomplished by making the volatile write: + +offset|content|before|after +---|---|---|--- +00-1A|irrelevant|| +1B|character|N|N +1C-1F|pointer|0000031E|_**0000039C**_ + +(Here and below normal writes are in bold and volatile writes in bold italic.) + +Readers have to read the pointer to reach the child (old or new), so this achieves the happens-before guarantees we +seek. Readers either see the old value (where none of the branch's data has been modified in any way), or the new value +(where the happens-before guarantees all writes creating the attached substructure are fully visible). + +Note that if the node is not the last in the chain, the pointer is implicit and we cannot change it. Thus we have +to copy, i.e. create a new node, which in this case will also be a `Chain` node, because there is nothing else in the +original node that needs to be preserved (the only existing transition is replaced by the update). + +#### Changing the child pointer of a `Sparse` or `Split` node + +Similarly to above, in this case the transition matches an existing one, and thus we already have a 4-byte location +where the pointer to the old child is written, and we can update it by doing a volatile write. + +For example, updating `C -> 0x51E` in a sparse node can be: + +offset|content|before|after +---|---|---|--- +00 - 03|child pointer 0| 00000238|00000238 +04 - 07|child pointer 1| 0000013A|_**0000051E**_ +08 - 0B|child pointer 2| 0000033B|0000033B +0C - 17|unused| +18 |character 0| 41 A|41 A +19 |character 1| 43 C|43 C +1A |character 2| 35 5|35 5 +1B - 1D|unused| +1E - 1F|order word| 0026 = 102 (base 6) + + +#### Adding a new child to `Split` + +If we already have the substructure that leads to the pointer for the new transition (i.e. a mid- and end-cell for the +transition's first 2-3 bits already exists), the situation is as above, where the existing pointer is `NONE`, and we can +simply perform a volatile write. + +If an end-cell mapping does not exist, we allocate a new cleared cell (so that all pointers are `NONE`), write the new +pointer at its position using a non-volatile write, and then create a mapping to this end-cell in the mid cell by +volatile writing its pointer over the `NONE` in the correct offset. Similarly, if there's no mid-cell either, we create +empty end-cell and mid-cell, write pointer in end-cell and mapping in mid-cell non-volatile, and write the mapping in +the leading cell volatile. + +In any of these cases, readers have to pass through the volatile update to reach any of the new content. + +For example, to add `x -> 0x71A` (`x` is `0x78` or `0b01111000`) to the split node example needs a new end cell for +`01 111` (for example at `0x720-0x73F`) (these writes can be non-volatile): + +offset|content|before|after +---|---|---|--- +00 - 03|pointer to child for ending 000|n/a|**0000071A** +04 - 07|pointer to child for ending 001|n/a|**00000000** NONE +08 - 0B|pointer to child for ending 010|n/a|**00000000** NONE +0C - 0F|pointer to child for ending 011|n/a|**00000000** NONE +10 - 13|pointer to child for ending 100|n/a|**00000000** NONE +14 - 17|pointer to child for ending 101|n/a|**00000000** NONE +18 - 1B|pointer to child for ending 110|n/a|**00000000** NONE +1C - 1F|pointer to child for ending 111|n/a|**00000000** NONE + +and this volatile write to the mid cell `0x520`: + +offset|content|before|after +---|---|---|--- +00 - 03|end-cell for middle 000|00000000 NONE|00000000 NONE +04 - 07|end-cell for middle 001|00000000 NONE|00000000 NONE +08 - 0B|end-cell for middle 010|00000000 NONE|00000000 NONE +0C - 0F|end-cell for middle 011|00000000 NONE|00000000 NONE +10 - 13|end-cell for middle 100|00000000 NONE|00000000 NONE +14 - 17|end-cell for middle 101|00000000 NONE|00000000 NONE +18 - 1B|end-cell for middle 110|00000540|00000540 +1C - 1F|end-cell for middle 111|00000000 NONE|_**00000720**_ + +The start cell, and the other mid and end cells remain unchanged. + +#### Adding a new child to `Sparse` with 5 or fewer existing children + +The need to maintain a correct view for concurrent readers without blocking is the reason why we cannot keep the +children in a `Sparse` cell ordered (if we insert ordered, we open ourselves to readers possibly seeing the same pointer +or child twice, or even going back in the iteration order). Instead we always add new characters and pointers at the +next free position and then update the order word to include it. More precisely: +- we find the smallest index `i < 6` for which the pointer is `NONE` +- we write the transition character at position `i` +- we write the pointer at position `i` over `NONE` volatile +- we compile a new order word by inserting `i` after all indexes with greater transition and before all indexes with + smaller in the base-6 representation (e.g. to insert `j` in sparse node that has `a@4 f@0 g@2 k@1 q@3` we change the + order word `31204` to `315204`) and write it volatile + +This ensures that any reader that iterates over children (i.e. one that needs the order word) will have to pass through +the volatile order word update and will see the correct character and pointer values. Readers who have read the order +word at some earlier time will not include the new pointer or character in the iteration. + +Readers that directly select the child for a given transition must read the pointer for each index _before_ reading the +character to ensure they can see the properly updated value (otherwise they could match e.g. a `00` transition to the +new branch because the real character was not written when they read the byte, but the pointer was when they got to it) +and stop searching when they find a `NONE` pointer. + +For example, adding `x -> 0x71A` to the sparse example above is done by: + +offset|content|before|after +---|---|---|--- +00 - 03|child pointer 0| 00000238|00000238 +04 - 07|child pointer 1| 0000051E|0000051E +08 - 0B|child pointer 2| 0000033B|0000033B +0C - 0F|child pointer 3|any|_**0000071A**_ +10 - 17|unused|NONE|NONE +18 |character 0| 41 A|41 A +19 |character 1| 43 C|43 C +1A |character 2| 35 5|35 5 +1B |character 3| any |**78** x +1C - 1D|unused|00 00|00 00 +1E - 1F|order word|0026 = 102 (base 6)|_**02AE**_ = 3102 (base 6) + +where we first write the character, then volatile write the pointer, and finally the order word. + +#### Changing the root pointer + +If an update propagates with copying modifications all the way to the root, we must update the root pointer. The latter +is a volatile variable, so this also enforces the happens-before relationship we need. + +### Merging a branch using `apply` + +This is a generalization of the mutation procedure above, which applies to more complex branches, where each node may +potentially need multiple updates to attach more than one child. The process proceeds as above; instead of keeping the +backtrack information in the call stack, we use the `Node.parentLink` pointers to point to `ApplyState` objects for +each node, which point to +- `mutationNode`, the node in the mutation trie, which contains a pointer up in its `parentLink` +- `existingNode`, the corresponding pointer in the memtable trie +- `updatedNode`, the current corresponding pointer in the memtable trie, which may be different from the above if the + mutation node is branching and one or more of its children have been already added + +When we descend, we follow the transitions in the memtable trie corresponding to the ones from an iteration over the +structure of the mutation trie to obtain the `existingNode` pointers, and initialize `updatedNode` to the same. When the +iteration processes a child, we apply the update to the node, which may happen in place, or may require copying +— in the latter case `updatedNode` will change to the new value. Note that if `updatedNode` was different from +the original `existingNode`, it was pointing to an unreachable copied node which will remain unreachable as we will only +attach the newer version. + +After all modifications coming as the result of application of child branches have been applied, we have an +`updatedNode` that reflects all. As we ascend we apply that new value to the parent's `updatedNode`. + +For example (adding a trie containing "traverse, truck" to the "tractor, tree, trie" one): + +![graph](MemtableTrie.md.a1.svg) + +In this diagram `existingNode`s are the ones reached through the light blue arrows during the descent phase (e.g. +`0x018` for the `ApplyState` at `tra`, or `NONE` for `tru`), and `updatedNode`s are the ones ascent (pink arrows) +returns with (e.g . `0x0DE` and `0x0FA` for the respective states). + +During this process, readers can see any modifications made in place (each in-place modification is an attachment point +which makes part of the new nodes reachable). The update mechanism above makes sure both that the state before the +update is preserved, and that the state after the update is fully visible for readers that can reach it, but it does not +guarantee that the mutation is seen atomically by the readers if it contains multiple separate branches. +It is possible for a reader to see only a part of the update, for example: +- a reading thread racing with the mutator can iterate over `traverse` but finish iterating before the mutator +manages to attach `truck`; +- a reading thread that iterated to `tree` (while `traverse` was not yet attached) and paused, will see `truck` if the +mutating thread applies the update during the pause. + +### Handling prefix nodes + +The descriptions above were given without prefix nodes. Handling prefixes is just a little complication over the update +process where we must augment `updatedNode` with any applicable content before applying the change to the parent. +To do this we expand the state tracked in `ApplyState` a little to: +- `existingPreContentNode` which points to the existing node including any prefix +- `existingPostContentNode` which is obtained by skipping over the prefix (for simplicity we also treat leaf + nodes like a prefix with no child) and is the base for all child updates (i.e. it takes the role of + `existingNode` in the descriptions above) +- `updatedPostContentNode` which is the node as changed/copied after children modifications are applied + +and we then apply the content (from existing prefix or newly introduced) to compile an `updatedPreContentNode` which +the parent is made to link to (which is equal to `updatedPostContentNode` if no content applies). + +("Pre-" and "post-" refer to descent/iteration order, not to construction order; e.g. `updatedPreContentNode` is +constructed after `updatedPostContentNode` but links above it in the trie.) + +As an example, consider the process of adding `trees` to our sample trie: + +![graph](MemtableTrie.md.p1.svg) + +When descending at `tree` we set `existingPreContentNode = ~1` and `existingPostContentNode = NONE`. Ascending back +to add the child `~3`, we add a child to `NONE` and get `updatedPostContentNode = 0x0BB`. To then apply the existing +content, we create the embedded prefix node `updatedPreContentNode = 0x0BF` with `contentIndex = 1` and pass that on to +the recursion. diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg new file mode 100644 index 000000000000..4237fb599a79 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.a1.svg @@ -0,0 +1,599 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + root->start + + + 0x09A + + + + t + + 0x09B + + + + root->t + + + t + + + + root->t + + + t + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tracto + + 0x01B + + + + tracto->tractor + + + r + + + + tract + + 0x01A + + + + tract->tracto + + + o + + + + trac + + 0x019 + + + + trac->tract + + + t + + + + trav + + Chain + 0x0B8 + + + + + tra + + Chain + 0x018 + + + + tra->trac + + + c + + + + tra->trav + + + v + + + + tra2 + + Sparse + 0x0DE + + + + + tree + + contentArray[1] + + + + trie + + contentArray[2] + + + + + tre + + Chain + 0x03B + + + + tre->tree + + + e + + + + tri + + Chain + 0x05B + + + + + truc + + 0x0FB + + + + + tri->trie + + + e + + + + tru + + Chain + 0x0FA + + + + + tr + + Sparse + 0x07E + + + + tr->tra + + + a + + + + tr->tra + + + a + + + + tr->tre + + + e + + + + tr->tri + + + i + + + + tr->t + + + 0x07E + + + + tr->tra2 + + + a + + + + tr->tru + + + u + + + + tr->tru + + + u + + + + t->root + + + 0x09B + + + + t->tr + + + r + + + + t->tr + + + r + + + + + trave + + 0x0B9 + + + + trav->trave + + + e + + + + trav->trave + + + e + + + + trav->tra2 + + + 0x0B8 + + + + trave->trav + + + 0x0B9 + + + + traver + + 0x0BA + + + + trave->traver + + + r + + + + trave->traver + + + r + + + + traver->trave + + + 0x0BA + + + + travers + + 0x0BB + + + + traver->travers + + + s + + + + traver->travers + + + s + + + + travers->traver + + + 0x0BB + + + + traverse + + contentArray[3] + + + + travers->traverse + + + e + + + + travers->traverse + + + e + + + + traverse->travers + + + ~3 + + + + tra2->trac + + + c + + + + + tra2->tr + + + 0x0DE + + + + tra2->trav + + + v + + + + tru->tr + + + 0x0FA + + + + tru->truc + + + c + + + + tru->truc + + + c + + + + truc->tru + + + 0x0FB + + + + truck + + contentArray[4] + + + + truc->truck + + + k + + + + truc->truck + + + k + + + + truck->truc + + + ~4 + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg new file mode 100644 index 000000000000..e43b324e2ba8 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g1.svg @@ -0,0 +1,76 @@ + + + + + + + + + G + + + + 0x13B + + 0x13B + + + + contentArray[0] + + contentArray[0] + + + + 0x13B->contentArray[0] + + +  A + + + + 0x13A + + 0x13A + + + + 0x13A->0x13B + + +  H + + + + 0x139 + + 0x139 + + + + 0x139->0x13A + + +  C + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg new file mode 100644 index 000000000000..a5c7eed6097b --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g2.svg @@ -0,0 +1,116 @@ + + + + + + + + + G + + + + 0x139 + + 0x139 + + + + 0x13A + + 0x13A + + + + 0x139->0x13A + + +  C + + + + 0x13B + + 0x13B + + + + contentArray[0] + + contentArray[0] + + + + 0x13B->contentArray[0] + + +  A + + + + 0x13A->0x13B + + +  H + + + + 0x25E + + 0x25E + + + + 0x25E->0x13A + + +  C + + + + 0x238 + + 0x238 + + + + 0x25E->0x238 + + +  A + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg new file mode 100644 index 000000000000..da6619c18547 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g3.svg @@ -0,0 +1,253 @@ + + + + + + + + + G + + + cluster_51C + + Split node 0x51C + + + + 0x51C + + 0x51C + + + + 0x520 + + 0x520 + + + + 0x51C->0x520 + + +  00 + + + + 0x560 + + 0x560 + + + + 0x51C->0x560 + + +  01 + + + + 0x540 + + 0x540 + + + + 0x520->0x540 + + +  110 + + + + 0x41B + + 0x41B + + + + 0x540->0x41B + + +  011 + + + + 0x33B + + 0x33B + + + + 0x540->0x33B + + +  101 + + + + 0x580 + + 0x580 + + + + 0x560->0x580 + + +  000 + + + + 0x5A0 + + 0x5A0 + + + + 0x560->0x5A0 + + +  010 + + + + 0x238 + + 0x238 + + + + 0x580->0x238 + + +  001 + + + + 0x35C + + 0x35C + + + + 0x580->0x35C + + +  010 + + + + 0x13A + + 0x13A + + + + 0x580->0x13A + + +  011 + + + + 0x37A + + 0x37A + + + + 0x5A0->0x37A + + +  000 + + + + 0x455 + + 0x455 + + + + 0x5A0->0x455 + + +  001 + + + + 0x13B + + 0x13B + + + + 0x13A->0x13B + + +  H + + + + contentArray[0] + + contentArray[0] + + + + 0x13B->contentArray[0] + + +  A + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg new file mode 100644 index 000000000000..d021029a7058 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.g4.svg @@ -0,0 +1,290 @@ + + + + + + + + + G + + + cluster_51F + + Node 0x51F + + + cluster_51C + + Split node 0x51C + + + + 0x13B + + 0x13B + + + + contentArray[0] + + contentArray[0] + + + + 0x13B->contentArray[0] + + +  A + + + + 0x13A + + 0x13A + + + + 0x13A->0x13B + + +  H + + + + 0x51F + + Prefix 0x51F + contentArray[1] + + + + 0x51C + + 0x51C + + + + 0x51F->0x51C + + + ε + + + + 0x520 + + 0x520 + + + + 0x51C->0x520 + + +  00 + + + + 0x560 + + 0x560 + + + + 0x51C->0x560 + + +  01 + + + + 0x540 + + 0x540 + + + + 0x520->0x540 + + +  110 + + + + 0x41B + + 0x41B + + + + 0x540->0x41B + + +  011 + + + + 0x33B + + 0x33B + + + + 0x540->0x33B + + +  101 + + + + 0x580 + + 0x580 + + + + 0x560->0x580 + + +  000 + + + + 0x5A0 + + 0x5A0 + + + + 0x560->0x5A0 + + +  010 + + + + 0x580->0x13A + + +  011 + + + + 0x238 + + 0x238 + + + + 0x580->0x238 + + +  001 + + + + 0x35C + + 0x35C + + + + 0x580->0x35C + + +  010 + + + + 0x37A + + 0x37A + + + + 0x5A0->0x37A + + +  000 + + + + 0x455 + + 0x455 + + + + 0x5A0->0x455 + + +  001 + + + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg new file mode 100644 index 000000000000..ff928a44dbec --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m1.svg @@ -0,0 +1,349 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + t + + 0x09B + + + + root->t + + + t + + + + root->t + + + t + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tracto + + 0x01B + + + + tracto->tractor + + + r + + + + tract + + 0x01A + + + + tract->tracto + + + o + + + + trac + + 0x019 + + + + trac->tract + + + t + + + + tree + + contentArray[1] + + + + + tra + + Chain + 0x018 + + + + tra->trac + + + c + + + + trav + + NONE + + + + tra->trav + + + v + + + + trie + + contentArray[2] + + + + + tre + + Chain + 0x03B + + + + tre->tree + + + e + + + + tri + + Chain + 0x05B + + + + tri->trie + + + e + + + + tr + + Sparse + 0x07E + + + + tr->tra + + + a + + + + tr->tra + + + a + + + + tr->tre + + + e + + + + tr->tri + + + i + + + + t->tr + + + r + + + + t->tr + + + r + + + + trave + + NONE + + + + trav->trave + + + e + + + + traver + + NONE + + + + trave->traver + + + r + + + + travers + + NONE + + + + traver->travers + + + s + + + + traverse + + NONE + + + + travers->traverse + + + e + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg new file mode 100644 index 000000000000..ba33dd1f2266 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m2.svg @@ -0,0 +1,430 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + t + + 0x09B + + + + root->t + + + t + + + + root->t + + + t + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tracto + + 0x01B + + + + tracto->tractor + + + r + + + + tract + + 0x01A + + + + tract->tracto + + + o + + + + trac + + 0x019 + + + + trac->tract + + + t + + + + tree + + contentArray[1] + + + + + tra + + Chain + 0x018 + + + + tra->trac + + + c + + + + trav + + 0x0B8 + + + + tra->trav + + + v + + + + trie + + contentArray[2] + + + + + tre + + Chain + 0x03B + + + + tre->tree + + + e + + + + tri + + Chain + 0x05B + + + + tri->trie + + + e + + + + tr + + Sparse + 0x07E + + + + tr->tra + + + a + + + + tr->tra + + + a + + + + tr->tre + + + e + + + + tr->tri + + + i + + + + t->tr + + + r + + + + t->tr + + + r + + + + trav->tra + + + 0x0B8 + + + + trave + + 0x0B9 + + + + trav->trave + + + e + + + + trav->trave + + + e + + + + trave->trav + + + 0x0B9 + + + + traver + + 0x0BA + + + + trave->traver + + + r + + + + trave->traver + + + r + + + + traver->trave + + + 0x0BA + + + + travers + + 0x0BB + + + + traver->travers + + + s + + + + traver->travers + + + s + + + + travers->traver + + + 0x0BB + + + + traverse + + contentArray[3] + + + + travers->traverse + + + e + + + + travers->traverse + + + e + + + + traverse->travers + + + ~3 + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg new file mode 100644 index 000000000000..e71114f89276 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.m3.svg @@ -0,0 +1,500 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + root->start + + + 0x09A + + + + t + + 0x09B + + + + root->t + + + t + + + + root->t + + + t + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tracto + + 0x01B + + + + tracto->tractor + + + r + + + + tract + + 0x01A + + + + tract->tracto + + + o + + + + trac + + 0x019 + + + + trac->tract + + + t + + + + trav + + Chain + 0x0B8 + + + + + tra + + Chain + 0x018 + + + + tra->trac + + + c + + + + tra->trav + + + v + + + + tra2 + + Sparse + 0x0DE + + + + + tree + + contentArray[1] + + + + trie + + contentArray[2] + + + + + tre + + Chain + 0x03B + + + + tre->tree + + + e + + + + tri + + Chain + 0x05B + + + + + tri->trie + + + e + + + + tr + + Sparse + 0x07E + + + + tr->tra + + + a + + + + tr->tra + + + a + + + + tr->tre + + + e + + + + tr->tri + + + i + + + + tr->t + + + 0x07E + + + + tr->tra2 + + + a + + + + t->root + + + 0x09B + + + + t->tr + + + r + + + + t->tr + + + r + + + + + trave + + 0x0B9 + + + + trav->trave + + + e + + + + trav->trave + + + e + + + + trav->tra2 + + + 0x0B8 + + + + trave->trav + + + 0x0B9 + + + + traver + + 0x0BA + + + + trave->traver + + + r + + + + trave->traver + + + r + + + + traver->trave + + + 0x0BA + + + + travers + + 0x0BB + + + + traver->travers + + + s + + + + traver->travers + + + s + + + + travers->traver + + + 0x0BB + + + + traverse + + contentArray[3] + + + + travers->traverse + + + e + + + + travers->traverse + + + e + + + + traverse->travers + + + ~3 + + + + tra2->trac + + + c + + + + + tra2->tr + + + 0x0DE + + + + tra2->trav + + + v + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg new file mode 100644 index 000000000000..c5268e71d64f --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.p1.svg @@ -0,0 +1,405 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + root->start + + + 0x09A + + + + t + + 0x09B + + + + root->t + + + t + + + + root->t + + + t + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tracto + + 0x01B + + + + tracto->tractor + + + r + + + + tract + + 0x01A + + + + tract->tracto + + + o + + + + trac + + 0x019 + + + + trac->tract + + + t + + + + tree + + contentArray[1] + + + + + tra + + Chain + 0x018 + + + + tra->trac + + + c + + + + tre + + Chain + 0x03B + + + + + tree2p + + Prefix + 0x0BF + contentArray[1] + + + + + trees + + contentArray[3] + + + + tree->trees + + + s + + + + tre->tree + + + e + + + + tre->tree + + + e + + + + tri + + Chain + 0x05B + + + + + tr + + Sparse + 0x07E + + + + tre->tr + + + 0x0DE + + + + tre->tree2p + + + e + + + + trie + + contentArray[2] + + + + tri->trie + + + e + + + + tr->tra + + + a + + + + tr->tre + + + e + + + + tr->tre + + + e + + + + tr->tri + + + i + + + + tr->t + + + 0x07E + + + + t->root + + + 0x09B + + + + t->tr + + + r + + + + t->tr + + + r + + + + tree2 + + Chain + 0x0BB + + + + tree2->tree2p + + + 0x0BB + + + + tree2->trees + + + s + + + + tree2p->tre + + + 0x0BF + + + + + tree2p->tree2 + + + ε + + + + trees->tree2 + + + ~3 + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg new file mode 100644 index 000000000000..1be94ae23739 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w1.svg @@ -0,0 +1,226 @@ + + + + + + + + + G + + + + tractor + + contentArray[0] + + + + tracto + + 0x01B + + + + tracto->tractor + + + r + + + + tract + + 0x01A + + + + tract->tracto + + + o + + + + trac + + 0x019 + + + + trac->tract + + + t + + + + tree + + contentArray[1] + + + + + tra + + Chain + 0x018 + + + + tra->trac + + + c + + + + trie + + contentArray[2] + + + + + tre + + Chain + 0x03B + + + + tre->tree + + + e + + + + tri + + Chain + 0x05B + + + + tri->trie + + + e + + + + tr + + Sparse + 0x07E + + + + tr->tra + + + a + + + + tr->tre + + + e + + + + tr->tri + + + i + + + + t + + 0x09B + + + + t->tr + + + r + + + + root + + Chain + 0x9A + + + + root->t + + + t + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg new file mode 100644 index 000000000000..9d8ab22e69a3 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w2.svg @@ -0,0 +1,326 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + root->start + + + + + + t + + 0x09B + + + + root->t + + + t + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tracto + + 0x01B + + + + tractor->tracto + + + + + + tracto->tractor + + + r + + + + tract + + 0x01A + + + + tracto->tract + + + + + + tract->tracto + + + o + + + + trac + + 0x019 + + + + tract->trac + + + + + + trac->tract + + + t + + + + tra + + Chain + 0x018 + + + + trac->tra + + + + + + tree + + contentArray[1] + + + + + tra->trac + + + c + + + + tr + + Sparse + 0x07E + + + + tra->tr + + + + + + tre + + Chain + 0x03B + + + + tree->tre + + + + + + trie + + contentArray[2] + + + + + tre->tree + + + e + + + + tre->tr + + + + + + tri + + Chain + 0x05B + + + + trie->tri + + + + + + tri->trie + + + e + + + + tri->tr + + + + + + tr->tra + + + a + + + + tr->tre + + + e + + + + tr->tri + + + i + + + + tr->t + + + + + + t->root + + + + + + t->tr + + + r + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg new file mode 100644 index 000000000000..626107a251a7 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w3.svg @@ -0,0 +1,276 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + tr + + Sparse + 0x07E + + + + root->tr + + + tr + + + + t + + 0x09B + + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tractor->tr + + + + + + tracto + + 0x01B + + + + + tract + + 0x01A + + + + + trac + + 0x019 + + + + + tree + + contentArray[1] + + + + + tra + + Chain + 0x018 + + + + tra->tractor + + + ctor + + + + + trie + + contentArray[2] + + + + + tree->tr + + + + + + tre + + Chain + 0x03B + + + + + tre->tree + + + e + + + + trie->tr + + + + + + tri + + Chain + 0x05B + + + + + tri->trie + + + e + + + + tr->start + + + + + + + tr->tra + + + a + + + + + tr->tre + + + e + + + + + tr->tri + + + i + + + + diff --git a/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w4.svg b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w4.svg new file mode 100644 index 000000000000..32bae7bd8175 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MemtableTrie.md.w4.svg @@ -0,0 +1,268 @@ + + + + + + + + + G + + + + root + + Chain + 0x9A + + + + start + + start/end + + + + + tr + + Sparse + 0x07E + + + + root->tr + + + tr + + + + t + + 0x09B + + + + + start->root + + + + + + tractor + + contentArray[0] + + + + tractor->tr + + + + + + tracto + + 0x01B + + + + + tract + + 0x01A + + + + + trac + + 0x019 + + + + + tree + + contentArray[1] + + + + + tra + + Chain + 0x018 + + + + tra->tractor + + + ctor + + + + + trie + + contentArray[2] + + + + + tree->tr + + + + + + tre + + Chain + 0x03B + + + + + tre->tree + + + e + + + + trie->start + + + + + + tri + + Chain + 0x05B + + + + + tri->trie + + + e + + + + + tr->tra + + + a + + + + + tr->tre + + + e + + + + + tr->tri + + + i + + + + diff --git a/src/java/org/apache/cassandra/db/tries/MergeTrie.java b/src/java/org/apache/cassandra/db/tries/MergeTrie.java new file mode 100644 index 000000000000..b8f7f512f670 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/MergeTrie.java @@ -0,0 +1,174 @@ +/* + * 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.tries; + +import com.google.common.collect.Iterables; + +/** + * A merged view of two tries. + */ +class MergeTrie extends Trie +{ + /** + * Transition value used to indicate a transition is not present. Must be greater than all valid transition values + * (0-0xFF). + */ + public static final int NOT_PRESENT = 0x100; + + private final MergeResolver resolver; + protected final Trie t1; + protected final Trie t2; + + MergeTrie(MergeResolver resolver, Trie t1, Trie t2) + { + this.resolver = resolver; + this.t1 = t1; + this.t2 = t2; + } + + @Override + protected Cursor cursor() + { + return new MergeCursor<>(resolver, t1, t2); + } + + static class MergeCursor implements Cursor + { + private final MergeResolver resolver; + private final Cursor c1; + private final Cursor c2; + + boolean atC1; + boolean atC2; + + MergeCursor(MergeResolver resolver, Trie t1, Trie t2) + { + this.resolver = resolver; + this.c1 = t1.cursor(); + this.c2 = t2.cursor(); + atC1 = atC2 = true; + } + + @Override + public int advance() + { + return checkOrder(atC1 ? c1.advance() : c1.level(), + atC2 ? c2.advance() : c2.level()); + } + + @Override + public int ascend() + { + int c1level = c1.level(); + int c2level = c2.level(); + int level = Math.max(c1level, c2level); + return checkOrder(c1level == level ? c1.ascend() : c1level, + c2level == level ? c2.ascend() : c2level); + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + if (atC1 & atC2) + return advance(); + + if (atC1) + { + int c2level = c2.level(); + int c1level = c1.advanceMultiple(receiver); + if (c1level <= c2level) + return checkOrder(c1level, c2level); + else + return c1level; // atC1 stays true, atC2 false, c2 remains where it is + } + else // atC2 + { + int c1level = c1.level(); + int c2level = c2.advanceMultiple(receiver); + if (c2level <= c1level) + return checkOrder(c1level, c2level); + else + return c2level; // atC2 stays true, atC1 false, c1 remains where it is + } + } + + private int checkOrder(int c1level, int c2level) + { + if (c1level > c2level) + { + atC1 = true; + atC2 = false; + return c1level; + } + if (c1level < c2level) + { + atC1 = false; + atC2 = true; + return c2level; + } + int c1trans = c1.incomingTransition(); + int c2trans = c2.incomingTransition(); + atC1 = c1trans <= c2trans; + atC2 = c1trans >= c2trans; + assert atC1 | atC2; + return c1level; + } + + @Override + public int level() + { + return atC1 ? c1.level() : c2.level(); + } + + @Override + public int incomingTransition() + { + return atC1 ? c1.incomingTransition() : c2.incomingTransition(); + } + + public T content() + { + T mc = atC2 ? c2.content() : null; + T nc = atC1 ? c1.content() : null; + if (mc == null) + return nc; + else if (nc == null) + return mc; + else + return resolver.resolve(nc, mc); + } + } + + /** + * Special instance for sources that are guaranteed (by the caller) distinct. The main difference is that we can + * form unordered value list by concatenating sources. + */ + static class Distinct extends MergeTrie + { + Distinct(Trie input1, Trie input2) + { + super(throwingResolver(), input1, input2); + } + + @Override + public Iterable valuesUnordered() + { + return Iterables.concat(t1.valuesUnordered(), t2.valuesUnordered()); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/RangeTrieSet.java b/src/java/org/apache/cassandra/db/tries/RangeTrieSet.java new file mode 100644 index 000000000000..166779c64d0d --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/RangeTrieSet.java @@ -0,0 +1,259 @@ +/* + * 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.tries; + +import java.util.Arrays; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + + +/** + * TrieSet representing the range between two keys. + * + * The keys must be correctly ordered, including with respect to the `includeLeft` and `includeRight` constraints. + * (i.e. RangeTrieSet(x, false, x, false) is an invalid call but RangeTrieSet(x, true, x, false) is inefficient + * but fine for an empty set). + */ +public class RangeTrieSet extends TrieSet +{ + /** Left-side boundary. The characters of this are requested as we descend along the left-side boundary. */ + private final ByteComparable left; + + /** Right-side boundary. The characters of this are requested as we descend along the right-side boundary. */ + private final ByteComparable right; + + private final boolean includeLeft; + private final boolean includeRight; + + RangeTrieSet(ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) + { + this.left = left; + this.includeLeft = includeLeft; + this.right = right; + this.includeRight = includeRight; + } + + protected Cursor cursor() + { + return new RangeCursor(this); + } + + private static class RangeCursor implements Cursor + { + private int[] backlog; + int backlogPos; + private ByteSource remainingLeftLimit; + private ByteSource remainingRightLimit; + boolean atLeftLimit; + boolean atRightLimit; + int leftLimitNext; + int rightLimitNext; + int transitionAtRightLevel; + private int incomingTransition; + private int level; + InSet inSet; + + + private RangeCursor(RangeTrieSet set) + { + backlog = new int[32]; + backlogPos = 0; + level = 0; + transitionAtRightLevel = -1; + if (set.left != null) + { + remainingLeftLimit = set.left.asComparableBytes(BYTE_COMPARABLE_VERSION); + if (!set.includeLeft) + remainingLeftLimit = ByteSource.nextKey(remainingLeftLimit); + leftLimitNext = remainingLeftLimit.next(); + atLeftLimit = leftLimitNext != ByteSource.END_OF_STREAM; + } + else + atLeftLimit = false; + + atRightLimit = set.right != null; + if (atRightLimit) + { + remainingRightLimit = set.right.asComparableBytes(BYTE_COMPARABLE_VERSION); + if (set.includeRight) + remainingRightLimit = ByteSource.nextKey(remainingRightLimit); + rightLimitNext = remainingRightLimit.next(); + if (rightLimitNext == ByteSource.END_OF_STREAM) + { + level = -1; + inSet = null; + return; + } + } + else + rightLimitNext = 256; + + incomingTransition = -1; + inSet = atLeftLimit ? null + : atRightLimit ? InSet.INCLUDED + : InSet.BRANCH; + } + + + public int advance() + { + if (atLeftLimit) + { + if (atRightLimit) + return descendAlongBoth(); + else + { + addBacklog(leftLimitNext + 1); + return descendAlongLeft(); + } + } + + if (processBacklog()) + return level; + + return continueAlongRight(); + } + + private int descendAlongBoth() + { + if (rightLimitNext > leftLimitNext) + { + atRightLimit = false; + transitionAtRightLevel = leftLimitNext + 1; + return descendAlongLeft(); + } + + assert rightLimitNext == leftLimitNext; + incomingTransition = leftLimitNext; + rightLimitNext = remainingRightLimit.next(); + leftLimitNext = remainingLeftLimit.next(); + if (leftLimitNext != ByteSource.END_OF_STREAM) + { + inSet = null; + assert rightLimitNext != ByteSource.END_OF_STREAM; + } + else + { + atLeftLimit = false; + if (rightLimitNext == ByteSource.END_OF_STREAM) + return -1; + + inSet = InSet.INCLUDED; + } + return ++level; + } + + private int descendAlongLeft() + { + int next = leftLimitNext; + leftLimitNext = remainingLeftLimit.next(); + + incomingTransition = next; + if (leftLimitNext != ByteSource.END_OF_STREAM) + { + inSet = null; + } + else + { + atLeftLimit = false; + inSet = InSet.BRANCH; + } + return ++level; + } + + private boolean processBacklog() + { + while (backlogPos > 0) + { + incomingTransition = backlog[backlogPos - 1]++; + if (incomingTransition < 256) + { + inSet = InSet.BRANCH; + return true; + } + --backlogPos; + --level; + } + return false; + } + + private int continueAlongRight() + { + if (transitionAtRightLevel < 0) + { + transitionAtRightLevel = 0; + ++level; + } + incomingTransition = transitionAtRightLevel++; + + if (incomingTransition < rightLimitNext) + { + inSet = InSet.BRANCH; + return level; + } + else + { + if (incomingTransition >= 256) // the no-right-limit case + return -1; + + rightLimitNext = remainingRightLimit.next(); + if (rightLimitNext == ByteSource.END_OF_STREAM) + return -1; + transitionAtRightLevel = -1; + inSet = InSet.INCLUDED; + return level; + } + } + + void addBacklog(int transition) + { + if (backlogPos == backlog.length) + backlog = Arrays.copyOf(backlog, backlogPos * 2); + backlog[backlogPos++] = transition; + } + + public int ascend() + { + atLeftLimit = false; + if (processBacklog()) + return level; + if (transitionAtRightLevel < 0) + return -1; + return continueAlongRight(); + } + + public int level() + { + return level; + } + + public int incomingTransition() + { + return incomingTransition; + } + + public InSet content() + { + return inSet; + } + } + + + // TODO: Change to start/stop sets when nodes are taken out of the picture +} diff --git a/src/java/org/apache/cassandra/db/tries/SetIntersectionTrie.java b/src/java/org/apache/cassandra/db/tries/SetIntersectionTrie.java new file mode 100644 index 000000000000..ab10e41a516c --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/SetIntersectionTrie.java @@ -0,0 +1,145 @@ +/* + * 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.tries; + +/** + * The intersection of a trie with the given set. + */ +class SetIntersectionTrie extends Trie +{ + private final Trie trie; + private final TrieSet intersectingSet; + + SetIntersectionTrie(Trie trie, TrieSet intersectingSet) + { + this.trie = trie; + this.intersectingSet = intersectingSet; + } + + protected Cursor cursor() + { + return new IntersectionCursor(trie.cursor(), intersectingSet.cursor()); + } + + private static class IntersectionCursor implements Cursor + { + private final Cursor tCursor; + private final Cursor sCursor; + + public IntersectionCursor(Cursor tCursor, + Cursor sCursor) + { + this.tCursor = tCursor; + this.sCursor = sCursor; + } + + @Override + public int advance() + { + int tLevel = tCursor.advance(); + if (sCursor.content() == TrieSet.InSet.BRANCH) + { + if (tLevel > sCursor.level()) + return tLevel; + // otherwise we have left the intersection's covered branch + } + int sLevel = sCursor.advance(); + + return advanceToIntersection(tLevel, sLevel); + } + + @Override + public int advanceMultiple(TransitionsReceiver transitionsReceiver) + { + int tLevel; + if (sCursor.content() == TrieSet.InSet.BRANCH) + { + tLevel = tCursor.advanceMultiple(transitionsReceiver); + if (tLevel > sCursor.level()) + return tLevel; + // otherwise we have left the intersection's covered branch + } + else + tLevel = tCursor.advance(); + + int sLevel = sCursor.advance(); + return advanceToIntersection(tLevel, sLevel); + } + + @Override + public int ascend() // this is not tested ATM + { + int tLevel = tCursor.ascend(); + if (sCursor.content() == TrieSet.InSet.BRANCH) + { + if (tLevel > sCursor.level()) + return tLevel; + // otherwise we have left the intersection's covered branch + } + int sLevel = sCursor.ascend(); + + return advanceToIntersection(tLevel, sLevel); + } + + private int advanceToIntersection(int tLevel, int sLevel) + { + while (sLevel != -1 && tLevel != -1) + { + if (sLevel == tLevel) + { + int tIncoming = tCursor.incomingTransition(); + int sIncoming = sCursor.incomingTransition(); + if (sIncoming == tIncoming) + return tLevel; // got entry + else if (sIncoming < tIncoming) + sLevel = sCursor.ascend(); + else // sIncoming > tIncoming + tLevel = tCursor.ascend(); + } + else if (sLevel < tLevel) + { + if (sCursor.content() == TrieSet.InSet.BRANCH) + return tLevel; + tLevel = tCursor.ascend(); + } + else // (sLevel > tLevel) + sLevel = sCursor.ascend(); + } + return -1; + } + + // TODO: implement advanceMultiple + + public int level() + { + return tCursor.level(); + } + + public int incomingTransition() + { + return tCursor.incomingTransition(); + } + + public T content() + { + return sCursor.content() != null + ? tCursor.content() + : null; + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/SingletonTrie.java b/src/java/org/apache/cassandra/db/tries/SingletonTrie.java new file mode 100644 index 000000000000..7445c030820b --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/SingletonTrie.java @@ -0,0 +1,98 @@ +/* + * 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.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/** + * Singleton trie, mapping the given key to value. + * Formed as a chain of single-child SNodes leading to one ENode with no children and the given value as content. + */ +class SingletonTrie extends Trie +{ + private final ByteComparable key; + private final T value; + + SingletonTrie(ByteComparable key, T value) + { + this.key = key; + this.value = value; + } + + public Cursor cursor() + { + return new Cursor(); + } + + class Cursor implements Trie.Cursor + { + ByteSource.Peekable src = ByteSource.peekable(key.asComparableBytes(BYTE_COMPARABLE_VERSION)); + int currentLevel = 0; + int currentTransition = -1; + + public int advance() + { + currentTransition = src.next(); + if (currentTransition != ByteSource.END_OF_STREAM) + return ++currentLevel; + else + return currentLevel = -1; + } + + @Override + public int advanceMultiple(TransitionsReceiver receiver) + { + int current = src.next(); + int level = currentLevel; + if (current == ByteSource.END_OF_STREAM) + return currentLevel = -1; + int next = src.next(); + while (next != ByteSource.END_OF_STREAM) + { + if (receiver != null) + receiver.add(current); + current = next; + next = src.next(); + ++level; + } + currentTransition = current; + return currentLevel = ++level; + } + + public int ascend() + { + return -1; // no alternatives + } + + public int level() + { + return currentLevel; + } + + public T content() + { + return src.peek() == ByteSource.END_OF_STREAM ? value : null; + } + + public int incomingTransition() + { + return currentTransition; + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/Trie.java b/src/java/org/apache/cassandra/db/tries/Trie.java new file mode 100644 index 000000000000..c9f64c60dec0 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/Trie.java @@ -0,0 +1,409 @@ +/* + * 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.tries; + +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.function.Function; + +import com.google.common.collect.ImmutableList; + +import org.agrona.concurrent.UnsafeBuffer; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * Base class for tries. + * + * Normal users of tries will only use the public transformation methods, which various transformations of the trie + * and conversion of its content to other formats (e.g. iterable of values). + * + * For any unimplemented data extraction operations one can rely on the TrieWalker (to aggregate value) and TrieIterator + * (to iterator) base classes, which provide the necessary mechanisms to handle walking the trie. + * + * The internal representation of tries using this interface is defined in the Node interface. + * + * Its design is largely defined by the requirement for iteratively retrieving content of the trie, for which it needs + * to be able to represent and save the state of any traversal efficiently, so that it can be preserved while a consumer + * is operating on an item. This also enables a possible future extension to support asynchronous retrieval of nodes. + * + * To enable that efficient state representation, the nodes that are used to retrieve the internal state of the trie are + * light stateful objects and always contain a link to some parent state. The role of parent state can often be played + * by the parent node, because its current state defines the transition that was taken to obtain the child, and it also + * has a reference to its own parent, effectively building a stack of nodes left to process each holding its own state. + * It is also possible to skip some levels of the descent in the state description, if e.g. there are no other paths to + * examine at those levels to continue the traversal (see getUniqueDescendant). Some traversal types may require more + * information (e.g. position in a character array or list of nodes being merged). The type of parent state link is + * defined by the consumer through the generic parameter L and it is supplied by the consumer as an argument to the + * getCurrentChild call -- that parent state is presented by the child in its parentLink field. + * + * + * To begin traversal over a trie, one must retrieve the root node of the trie by calling root(). Because the nodes are + * stateful, the traversal must always proceed from one thread. Should concurrent reads be required, separate calls to + * root() must be made. + * + * Once a node is available, one can retrieve any associated content and list the children of the node along with their + * associated transition byte by: + * - calling startIteration to set the node's state to its first child + * - retrieving the associated transition byte using the node's currentTransition field + * - optionally retrieving the child using getCurrentChild giving it something you can use to restore your state + * to continue processing the rest of the children of this node + * - when processing the child is complete/skipped or child is null, request the next child using advanceIteration and + * repeat + * - if start/advanceIteration return null, there are no further children of the node + * - if they return Remaining.ONE, this is the last child of the node (the inverse is not always true, nodes will try + * but do not guarantee they will report ONE on their last child) + * - when the children are exhausted, use the node's parent link to restore your state to what it was when the relevant + * parent was being processed + * For an example of simple traversal, see TrieWalker. For a more complex traversal example, refer to TrieValuesIterator + * and TrieEntriesIterator. + * + * Note: This model only supports depth-first traversals. We do not currently have a need for breadth-first walks. + * + * @param The content type of the trie. Content is only allowed on leaf nodes. + */ +public abstract class Trie +{ + /** + * Enum used to indicate the presence of more children during the iteration of a node. + * Generally iteration will return null or MULTIPLE, but it can return ONE if it is known that there are no further + * children to optimize walks. + */ + protected enum Remaining + { + ONE, MULTIPLE + } + + /** + * Used by {@link Cursor#advanceMultiple} to feed the transitions taken. + */ + protected interface TransitionsReceiver + { + /** Add a single byte to the path. */ + void add(int t); + /** Add the count bytes from position pos at the given buffer. */ + void add(UnsafeBuffer b, int pos, int count); + } + + /** + * Used by {@link Cursor#advanceToContent} to track the transitions and backtracking taken. + */ + interface ResettingTransitionsReceiver extends TransitionsReceiver + { + /** Delete all bytes beyond the given length. */ + void reset(int newLength); + } + + // Cursor-style walks + interface Cursor + { + /** + * Advance one position. + * This can be either: + * - descending one level + * - ascending to the closest parent that has remaining children, and then descending one level + * @return level (can be prev+1 or <=prev), -1 means done + */ + int advance(); + + /** + * Advance, descending multiple levels if that does not require extra work (e.g. chain nodes) + * Receiver will be given all transitions taken except the last; i.e. on an ascend it will not receive any + * + * @param receiver + * @return + */ + default int advanceMultiple(TransitionsReceiver receiver) + { + return advance(); + } + + default T advanceToContent(ResettingTransitionsReceiver receiver) // advances all the way (to next content) + { + int prevLevel = level(); + while (true) + { + int currLevel = advanceMultiple(receiver); + if (currLevel <= 0) + return null; + if (receiver != null) + { + if (currLevel <= prevLevel) + receiver.reset(currLevel - 1); + receiver.add(incomingTransition()); + } + T content = content(); + if (content != null) + return content; + prevLevel = currLevel; + } + } + + /** + * ignore the remaining children at this level or below and ascend to parent and advance + */ + int ascend(); // ignore the remaining children at this level or below and ascend to parent and advance + + int level(); // return current state; if just starting / on root, return 0 + int incomingTransition(); // return the last transition taken; if just starting / on root, return -1 + T content(); // return content -- may be non-null on root + + } + + protected abstract Cursor cursor(); + + // Version of the byte comparable conversion to use for all operations + static final ByteComparable.Version BYTE_COMPARABLE_VERSION = ByteComparable.Version.OSS41; + + public String dump() + { + return dump(Object::toString); + } + + public String dump(Function contentToString) + { + return TrieDumper.process(contentToString, this); + } + + /** + * Returns a singleton trie mapping the given byte path to content. + */ + public static Trie singleton(ByteComparable b, T v) + { + return new SingletonTrie<>(b, v); + } + + /** + * Returns a view of the subtrie containing everything in this trie whose keys fall between the given boundaries. + * + * This method will throw an assertion error if the bounds provided are not correctly ordered, including with + * respect to the `includeLeft` and `includeRight` constraints (i.e. subtrie(x, false, x, false) is an invalid call + * but subtrie(x, true, x, false) is inefficient but fine for an empty subtrie). + * + * @param left the left bound for the returned subtrie. If {@code null}, the resulting subtrie is not left-bounded. + * @param includeLeft whether {@code left} is an inclusive bound of not. + * @param right the right bound for the returned subtrie. If {@code null}, the resulting subtrie is not right-bounded. + * @param includeRight whether {@code right} is an inclusive bound of not. + * @return a view of the subtrie containing all the keys of this trie falling between {@code left} (inclusively if + * {@code includeLeft}) and {@code right} (inclusively if {@code includeRight}). + */ + public Trie subtrie(ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) + { + if (left == null && right == null) + return this; + + return new SetIntersectionTrie<>(this, TrieSet.range(left, includeLeft, right, includeRight)); + } + + /** + * Returns the ordered entry set of this trie's content as an iterable. + */ + public Iterable> entrySet() + { + return this::entryIterator; + } + + /** + * Returns the ordered entry set of this trie's content in an iterator. + */ + public Iterator> entryIterator() + { + return new TrieEntriesIterator.AsEntries<>(this); + } + + /** + * Returns the ordered set of values of this trie as an iterable. + */ + public Iterable values() + { + + return this::valueIterator; + } + + /** + * Returns the ordered set of values of this trie in an iterator. + */ + public Iterator valueIterator() + { + return new TrieValuesIterator<>(this); + } + + /** + * Returns the values in any order. For some tries this is much faster than the ordered iterable. + */ + public Iterable valuesUnordered() + { + return values(); + } + + /** + * Resolver of content of merged nodes, used for two-source merges (i.e. mergeWith). + */ + public interface MergeResolver + { + // Note: No guarantees about argument order. + // E.g. during t1.mergeWith(t2, resolver), resolver may be called with t1 or t2's items as first argument. + T resolve(T b1, T b2); + } + + /** + * Constructs a view of the merge of this trie with the given one. The view is live, i.e. any write to any of the + * sources will be reflected in the merged view. + * + * If there is content for a given key in both sources, the resolver will be called to obtain the combination. + * (The resolver will not be called if there's content from only one source.) + */ + public Trie mergeWith(Trie other, MergeResolver resolver) + { + return new MergeTrie<>(resolver, this, other); + } + + /** + * Resolver of content of merged nodes. + * + * The resolver's methods are only called if more than one of the merged nodes contain content, and the + * order in which the arguments are given is not defined. Only present non-null values will be included in the + * collection passed to the resolving methods. + * + * Can also be used as a two-source resolver. + */ + public interface CollectionMergeResolver extends MergeResolver + { + T resolve(Collection contents); + + default T resolve(T c1, T c2) + { + return resolve(ImmutableList.of(c1, c2)); + } + } + + private static final CollectionMergeResolver THROWING_RESOLVER = new CollectionMergeResolver() + { + public Object resolve(Collection contents) + { + throw error(); + } + + private AssertionError error() + { + throw new AssertionError("Entries must be distinct."); + } + }; + + /** + * Returns a resolver that throws whenever more than one of the merged nodes contains content. + * Can be used to merge tries that are known to have distinct content paths. + */ + public static CollectionMergeResolver throwingResolver() + { + return (CollectionMergeResolver) THROWING_RESOLVER; + } + + /** + * Constructs a view of the merge of multiple tries. The view is live, i.e. any write to any of the + * sources will be reflected (eventually consistently) in the merged view. + * + * If there is content for a given key in more than one sources, the resolver will be called to obtain the combination. + * (The resolver will not be called if there's content from only one source.) + */ + public static Trie merge(Collection> sources, CollectionMergeResolver resolver) + { + switch (sources.size()) + { + case 0: + return empty(); + case 1: + return sources.iterator().next(); + case 2: + { + Iterator> it = sources.iterator(); + Trie t1 = it.next(); + Trie t2 = it.next(); + return t1.mergeWith(t2, resolver); + } + default: + return new CollectionMergeTrie<>(sources, resolver); + } + } + + /** + * Constructs a view of the merge of multiple tries, where each source must have distinct keys. The view is live, + * i.e. any write to any of the sources will be reflected in the merged view. + * + * If there is content for a given key in more than one sources, the merge will throw an assertion error. + */ + public static Trie mergeDistinct(Collection> sources) + { + switch (sources.size()) + { + case 0: + return empty(); + case 1: + return sources.iterator().next(); + case 2: + { + Iterator> it = sources.iterator(); + Trie t1 = it.next(); + Trie t2 = it.next(); + return new MergeTrie.Distinct<>(t1, t2); + } + default: + return new CollectionMergeTrie.Distinct<>(sources); + } + } + + private static final Trie EMPTY = new Trie() + { + protected Cursor cursor() + { + return new Cursor() + { + public int advance() + { + return -1; + } + + public int ascend() + { + return -1; + } + + public int level() + { + return -1; + } + + public Object content() + { + return null; + } + + public int incomingTransition() + { + return -1; + } + }; + } + }; + + @SuppressWarnings("unchecked") + public static Trie empty() + { + return (Trie) EMPTY; + } +} diff --git a/src/java/org/apache/cassandra/db/tries/TrieDumper.java b/src/java/org/apache/cassandra/db/tries/TrieDumper.java new file mode 100644 index 000000000000..d1f06a1c951d --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieDumper.java @@ -0,0 +1,93 @@ +/* + * 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.tries; + +import java.util.function.Function; + +import org.agrona.concurrent.UnsafeBuffer; + +/** + * Simple utility class for dumping the structure of a trie to string. + */ +class TrieDumper +{ + public static String process(Function contentToString, Trie trie) + { + return dump(contentToString, trie.cursor()); + } + + static String dump(Function contentToString, Trie.Cursor cursor) + { + StringBuilder sb = new StringBuilder(); + Trie.ResettingTransitionsReceiver receiver = new TransitionsDumper(sb); + T content = cursor.content(); + if (content == null) + content = cursor.advanceToContent(receiver); + while (content != null) + { + sb.append(" -> "); + sb.append(contentToString.apply(content)); + receiver.reset(cursor.level()); + content = cursor.advanceToContent(receiver); + } + return sb.toString(); + } + + private static class TransitionsDumper implements Trie.ResettingTransitionsReceiver + { + private final StringBuilder b; + int needsIndent = -1; + + public TransitionsDumper(StringBuilder b) + { + this.b = b; + } + + @Override + public void reset(int newLength) + { + needsIndent = newLength; + } + + private void maybeIndent() + { + if (needsIndent >= 0) + { + b.append('\n'); + for (int i = 0; i < needsIndent; ++i) + b.append(" "); + needsIndent = -1; + } + } + + @Override + public void add(int incomingTransition) + { + maybeIndent(); + b.append(String.format("%02x", incomingTransition)); + } + + @Override + public void add(UnsafeBuffer buf, int pos, int count) + { + maybeIndent(); + for (int i = 0; i < count; ++i) + b.append(String.format("%02x", buf.getByte(pos + i) & 0xFF)); + } + } +} diff --git a/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java new file mode 100644 index 000000000000..24a84a353882 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieEntriesIterator.java @@ -0,0 +1,111 @@ +/* + * 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.tries; + +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Map; + +import org.agrona.concurrent.UnsafeBuffer; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * Convertor of trie entries to iterator where each entry is passed through {@link #mapContent} (to be implemented by + * descendants). + */ +public abstract class TrieEntriesIterator implements Iterator, Trie.ResettingTransitionsReceiver +{ + private final Trie.Cursor cursor; + private byte[] keyBytes = new byte[32]; + private int keyPos = 0; + T next; + boolean gotNext; + + protected TrieEntriesIterator(Trie trie) + { + cursor = trie.cursor(); + next = cursor.content(); + gotNext = next != null; + } + + public boolean hasNext() + { + if (!gotNext) + { + next = cursor.advanceToContent(this); + gotNext = true; + } + + return next != null; + } + + public V next() + { + gotNext = false; + T v = next; + next = null; + return mapContent(v, keyBytes, keyPos); + } + + public void add(int t) + { + if (keyPos >= keyBytes.length) + keyBytes = Arrays.copyOf(keyBytes, keyPos * 2); + keyBytes[keyPos++] = (byte) t; + } + + public void add(UnsafeBuffer b, int pos, int count) + { + int newPos = keyPos + count; + if (newPos > keyBytes.length) + keyBytes = Arrays.copyOf(keyBytes, Math.max(newPos + 16, keyBytes.length * 2)); + b.getBytes(pos, keyBytes, keyPos, count); + keyPos = newPos; + } + + public void reset(int newLength) + { + keyPos = newLength; + } + + protected abstract V mapContent(T content, byte[] bytes, int byteLength); + + /** + * Iterator representing the content of the trie a sequence of (path, content) pairs. + */ + static class AsEntries + extends TrieEntriesIterator> + { + public AsEntries(Trie trie) + { + super(trie); + } + + protected Map.Entry mapContent(T content, byte[] bytes, int byteLength) + { + return toEntry(content, bytes, byteLength); + } + } + + static java.util.Map.Entry toEntry(T content, byte[] bytes, int byteLength) + { + ByteComparable b = ByteComparable.fixedLength(Arrays.copyOf(bytes, byteLength)); + return new AbstractMap.SimpleImmutableEntry<>(b, content); + } +} diff --git a/src/java/org/apache/cassandra/db/tries/TrieSet.java b/src/java/org/apache/cassandra/db/tries/TrieSet.java new file mode 100644 index 000000000000..2a7fce45c138 --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieSet.java @@ -0,0 +1,99 @@ +/* + * 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.tries; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * A simplified version of Trie used for sets (whose ultimate function is to intersect a Trie). + * + * Sets cannot be asynchronous and support a special value to denote a branch is fully included in the set, + * which is used to speed up intersections. + * + * Like Trie nodes, set nodes are stateful and not thread-safe. If the consumer can use multiple threads when accessing + * a node (e.g. with asynchronous trie walks), it must enforce a happens-before relationship between calls to the + * methods of a node. + */ +public abstract class TrieSet extends Trie +{ + enum InSet + { + // null content value specifies that the specific point is not contained in the set (e.g. points on the left range path) + INCLUDED, // this point is contained in the set (e.g. points on the right range path) + BRANCH; // the whole branch is contained in the set (e.g. interior nodes for a range) + } + + private static final TrieSet FULL_SET = new TrieSet() + { + @Override + protected Cursor cursor() + { + return new Cursor() + { + int level = 0; + + @Override + public int advance() + { + return level = -1; + } + + @Override + public int ascend() + { + return level = -1; + } + + @Override + public int level() + { + return level; + } + + @Override + public int incomingTransition() + { + return -1; + } + + @Override + public InSet content() + { + return InSet.BRANCH; + } + }; + } + }; + + /** + * Range of keys between the given boundaries. + * A null argument for any of the limits means that the set should be unbounded on that side. + * The keys must be correctly ordered, including with respect to the `includeLeft` and `includeRight` constraints. + * (i.e. range(x, false, x, false) is an invalid call but range(x, true, x, false) is inefficient + * but fine for an empty set). + */ + public static TrieSet range(ByteComparable left, boolean includeLeft, ByteComparable right, boolean includeRight) + { + return new RangeTrieSet(left, includeLeft, right, includeRight); + } + + public static TrieSet full() + { + return FULL_SET; + } +} diff --git a/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java b/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java new file mode 100644 index 000000000000..7297ee12d1bc --- /dev/null +++ b/src/java/org/apache/cassandra/db/tries/TrieValuesIterator.java @@ -0,0 +1,60 @@ +/* + * 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.tries; + +import java.util.Iterator; + +/** + * Convertor of trie contents to flow. + * + * Note: the type argument L must be equal to {@code Trie.Node}, but we can't define such a recursive type in + * Java. Using {@code <>} when instantiating works, but any subclasses will also need to declare this useless type + * argument. + */ +class TrieValuesIterator implements Iterator +{ + private final Trie.Cursor cursor; + T next; + boolean gotNext = false; + + protected TrieValuesIterator(Trie trie) + { + cursor = trie.cursor(); + next = cursor.content(); + gotNext = next != null; + } + + public boolean hasNext() + { + if (!gotNext) + { + next = cursor.advanceToContent(null); + gotNext = true; + } + + return next != null; + } + + public T next() + { + gotNext = false; + T v = next; + next = null; + return v; + } +} diff --git a/src/java/org/apache/cassandra/db/view/TableViews.java b/src/java/org/apache/cassandra/db/view/TableViews.java index cc58dc19e740..421aa44719ba 100644 --- a/src/java/org/apache/cassandra/db/view/TableViews.java +++ b/src/java/org/apache/cassandra/db/view/TableViews.java @@ -102,16 +102,16 @@ public void stopBuild() views.forEach(View::stopBuild); } - public void forceBlockingFlush() + public void forceBlockingFlush(ColumnFamilyStore.FlushReason reason) { for (ColumnFamilyStore viewCfs : allViewsCfs()) - viewCfs.forceBlockingFlush(); + viewCfs.forceBlockingFlush(reason); } - public void dumpMemtables() + public void dumpMemtables(ColumnFamilyStore.FlushReason reason) { for (ColumnFamilyStore viewCfs : allViewsCfs()) - viewCfs.dumpMemtable(); + viewCfs.dumpMemtable(reason); } public void truncateBlocking(CommitLogPosition replayAfter, long truncatedAt) diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java index d813d0e66885..cc7941a18099 100644 --- a/src/java/org/apache/cassandra/db/view/View.java +++ b/src/java/org/apache/cassandra/db/view/View.java @@ -35,6 +35,7 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.schema.ViewMetadata; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.utils.FBUtilities; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -201,7 +202,7 @@ private List selectClause() ReadQuery getReadQuery() { if (query == null) - query = getSelectStatement().getQuery(QueryOptions.forInternalCalls(Collections.emptyList()), FBUtilities.nowInSeconds()); + query = getSelectStatement().getQuery(QueryState.forInternalCalls(), QueryOptions.forInternalCalls(Collections.emptyList()), FBUtilities.nowInSeconds()); return query; } diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java index a88ffbecc3fe..27ff53118f7b 100644 --- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java +++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java @@ -96,7 +96,7 @@ public void start() logger.debug("Starting build of view({}.{}). Flushing base table {}.{}", ksName, view.name, ksName, baseCfs.name); - baseCfs.forceBlockingFlush(); + baseCfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.VIEW_BUILD_STARTED); loadStatusAndBuild(); } diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java b/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java index c84c6978dcce..3739b325aba8 100644 --- a/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java +++ b/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java @@ -29,7 +29,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Objects; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; import com.google.common.util.concurrent.Futures; @@ -44,8 +43,7 @@ import org.apache.cassandra.db.ReadQuery; import org.apache.cassandra.db.SinglePartitionReadCommand; import org.apache.cassandra.db.SystemKeyspace; -import org.apache.cassandra.db.compaction.CompactionInfo; -import org.apache.cassandra.db.compaction.CompactionInfo.Unit; +import org.apache.cassandra.db.compaction.AbstractTableOperation; import org.apache.cassandra.db.compaction.CompactionInterruptedException; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.lifecycle.SSTableSet; @@ -63,7 +61,7 @@ import org.apache.cassandra.utils.UUIDGen; import org.apache.cassandra.utils.concurrent.Refs; -public class ViewBuilderTask extends CompactionInfo.Holder implements Callable +public class ViewBuilderTask extends AbstractTableOperation implements Callable { private static final Logger logger = LoggerFactory.getLogger(ViewBuilderTask.class); @@ -191,12 +189,12 @@ private void finish() // If it's stopped due to a compaction interruption we should throw that exception. // Otherwise we assume that the task has been stopped due to a schema update and we can finish successfully. if (isCompactionInterrupted) - throw new StoppedException(ksName, view.name, getCompactionInfo()); + throw new StoppedException(ksName, view.name, getProgress()); } } @Override - public CompactionInfo getCompactionInfo() + public OperationProgress getProgress() { // we don't know the sstables at construction of ViewBuilderTask and we could change this to return once we know the // but since we basically only cancel view builds on truncation where we cancel all compactions anyway, this seems reasonable @@ -205,13 +203,13 @@ public CompactionInfo getCompactionInfo() if (range.left.getPartitioner().splitter().isPresent()) { long progress = prevToken == null ? 0 : Math.round(prevToken.getPartitioner().splitter().get().positionInRange(prevToken, range) * 1000); - return CompactionInfo.withoutSSTables(baseCfs.metadata(), OperationType.VIEW_BUILD, progress, 1000, Unit.RANGES, compactionId); + return OperationProgress.withoutSSTables(baseCfs.metadata(), OperationType.VIEW_BUILD, progress, 1000, Unit.RANGES, compactionId); } // When there is no splitter, estimate based on number of total keys but // take the max with keysBuilt + 1 to avoid having more completed than total long keysTotal = Math.max(keysBuilt + 1, baseCfs.estimatedKeysForRange(range)); - return CompactionInfo.withoutSSTables(baseCfs.metadata(), OperationType.VIEW_BUILD, keysBuilt, keysTotal, Unit.KEYS, compactionId); + return OperationProgress.withoutSSTables(baseCfs.metadata(), OperationType.VIEW_BUILD, keysBuilt, keysTotal, Unit.KEYS, compactionId); } @Override @@ -248,7 +246,7 @@ static class StoppedException extends CompactionInterruptedException { private final String ksName, viewName; - private StoppedException(String ksName, String viewName, CompactionInfo info) + private StoppedException(String ksName, String viewName, OperationProgress info) { super(info); this.ksName = ksName; diff --git a/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java b/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java index 20033dfed4cb..4ddb73201870 100644 --- a/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java +++ b/src/java/org/apache/cassandra/db/virtual/SSTableTasksTable.java @@ -17,7 +17,7 @@ */ package org.apache.cassandra.db.virtual; -import org.apache.cassandra.db.compaction.CompactionInfo; +import org.apache.cassandra.db.compaction.AbstractTableOperation; import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.marshal.DoubleType; import org.apache.cassandra.db.marshal.LongType; @@ -58,21 +58,21 @@ public DataSet data() { SimpleDataSet result = new SimpleDataSet(metadata()); - for (CompactionInfo task : CompactionManager.instance.getSSTableTasks()) + for (AbstractTableOperation.OperationProgress task : CompactionManager.instance.getSSTableTasks()) { - long completed = task.getCompleted(); - long total = task.getTotal(); + long completed = task.completed(); + long total = task.total(); double completionRatio = total == 0L ? 1.0 : (((double) completed) / total); - result.row(task.getKeyspace().orElse("*"), - task.getTable().orElse("*"), - task.getTaskId()) + result.row(task.keyspace().orElse("*"), + task.table().orElse("*"), + task.operationId()) .column(COMPLETION_RATIO, completionRatio) - .column(KIND, task.getTaskType().toString().toLowerCase()) + .column(KIND, task.operationType().toString().toLowerCase()) .column(PROGRESS, completed) .column(TOTAL, total) - .column(UNIT, task.getUnit().toString().toLowerCase()); + .column(UNIT, task.unit().toString().toLowerCase()); } return result; diff --git a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java index 92da4afda53e..fe7591c1b979 100644 --- a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java @@ -19,6 +19,10 @@ import com.google.common.collect.ImmutableList; +import org.apache.cassandra.index.sai.virtual.IndexesSystemView; +import org.apache.cassandra.index.sai.virtual.SSTablesSystemView; +import org.apache.cassandra.index.sai.virtual.SegmentsSystemView; + import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_VIEWS; public final class SystemViewsKeyspace extends VirtualKeyspace @@ -36,6 +40,9 @@ private SystemViewsKeyspace() .add(new ThreadPoolsTable(VIRTUAL_VIEWS)) .add(new InternodeOutboundTable(VIRTUAL_VIEWS)) .add(new InternodeInboundTable(VIRTUAL_VIEWS)) + .add(new SSTablesSystemView(VIRTUAL_VIEWS)) + .add(new SegmentsSystemView(VIRTUAL_VIEWS)) + .add(new IndexesSystemView(VIRTUAL_VIEWS)) .addAll(TableMetricTables.getAll(VIRTUAL_VIEWS)) .build()); } diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java index a6314dcccc8e..2b0e2a286147 100644 --- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java +++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java @@ -26,6 +26,9 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Hex; import org.apache.cassandra.utils.ObjectSizes; @@ -37,7 +40,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; @@ -102,6 +104,12 @@ public boolean equals(Object obj) return Arrays.equals(token, other.token); } + @Override + public ByteSource asComparableBytes(ByteComparable.Version version) + { + return ByteSource.of(token, version); + } + @Override public IPartitioner getPartitioner() { @@ -223,6 +231,11 @@ public BytesToken getRandomToken(Random random) private final Token.TokenFactory tokenFactory = new Token.TokenFactory() { + public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return new BytesToken(ByteSourceInverse.getUnescapedBytes(comparableBytes)); + } + public ByteBuffer toByteArray(Token token) { BytesToken bytesToken = (BytesToken) token; diff --git a/src/java/org/apache/cassandra/dht/LocalPartitioner.java b/src/java/org/apache/cassandra/dht/LocalPartitioner.java index 168601ca3ef8..d69b8cd45493 100644 --- a/src/java/org/apache/cassandra/dht/LocalPartitioner.java +++ b/src/java/org/apache/cassandra/dht/LocalPartitioner.java @@ -26,7 +26,10 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.CachedHashDecoratedKey; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.memory.HeapAllocator; @@ -83,6 +86,12 @@ public Token.TokenFactory getTokenFactory() private final Token.TokenFactory tokenFactory = new Token.TokenFactory() { + public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + ByteBuffer tokenData = comparator.fromComparableBytes(ByteBufferAccessor.instance, comparableBytes, version); + return new LocalToken(tokenData); + } + public ByteBuffer toByteArray(Token token) { return ((LocalToken)token).token; @@ -174,6 +183,12 @@ public boolean equals(Object obj) return token.equals(other.token); } + @Override + public ByteSource asComparableBytes(ByteComparable.Version version) + { + return comparator.asComparableBytes(ByteBufferAccessor.instance, token, version); + } + @Override public IPartitioner getPartitioner() { diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java index 2856f131f1ab..dc5ee1308332 100644 --- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java +++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java @@ -33,6 +33,9 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.apache.cassandra.utils.MurmurHash; import org.apache.cassandra.utils.ObjectSizes; @@ -176,6 +179,12 @@ public int compareTo(Token o) return Long.compare(token, ((LongToken) o).token); } + @Override + public ByteSource asComparableBytes(ByteComparable.Version version) + { + return ByteSource.of(token); + } + @Override public IPartitioner getPartitioner() { @@ -194,6 +203,12 @@ public Object getTokenValue() return token; } + @Override + public long getLongValue() + { + return token; + } + @Override public double size(Token next) { @@ -316,6 +331,12 @@ public Token.TokenFactory getTokenFactory() private final Token.TokenFactory tokenFactory = new Token.TokenFactory() { + public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + long tokenData = ByteSourceInverse.getSignedLong(comparableBytes); + return new LongToken(tokenData); + } + public ByteBuffer toByteArray(Token token) { LongToken longToken = (LongToken) token; diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java index 16c5db17a448..cc55b10de620 100644 --- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java +++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java @@ -33,6 +33,9 @@ import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.Pair; @@ -128,6 +131,11 @@ public StringToken getRandomToken(Random random) private final Token.TokenFactory tokenFactory = new Token.TokenFactory() { + public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return new StringToken(ByteSourceInverse.getString(comparableBytes)); + } + public ByteBuffer toByteArray(Token token) { StringToken stringToken = (StringToken) token; @@ -194,6 +202,12 @@ public long getHeapSize() { return EMPTY_SIZE + ObjectSizes.sizeOf(token); } + + @Override + public ByteSource asComparableBytes(ByteComparable.Version version) + { + return ByteSource.of((String) token, version); + } } public StringToken getToken(ByteBuffer key) diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java index 241b7850fdf7..d02cfd58adfd 100644 --- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java +++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java @@ -27,6 +27,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.cassandra.db.CachedHashDecoratedKey; +import org.apache.cassandra.db.marshal.ByteArrayAccessor; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.marshal.AbstractType; @@ -34,6 +36,8 @@ import org.apache.cassandra.db.marshal.PartitionerDefinedOrder; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.GuidGenerator; import org.apache.cassandra.utils.ObjectSizes; @@ -158,6 +162,11 @@ private boolean isValidToken(BigInteger token) { private final Token.TokenFactory tokenFactory = new Token.TokenFactory() { + public Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version) + { + return fromByteArray(IntegerType.instance.fromComparableBytes(ByteBufferAccessor.instance, comparableBytes, version)); + } + public ByteBuffer toByteArray(Token token) { BigIntegerToken bigIntegerToken = (BigIntegerToken) token; @@ -244,6 +253,12 @@ public BigIntegerToken(String token) this(new BigInteger(token)); } + @Override + public ByteSource asComparableBytes(ByteComparable.Version version) + { + return IntegerType.instance.asComparableBytes(ByteArrayAccessor.instance, token.toByteArray(), version); + } + @Override public IPartitioner getPartitioner() { diff --git a/src/java/org/apache/cassandra/dht/Range.java b/src/java/org/apache/cassandra/dht/Range.java index 5b2f3d9fbf16..50cf4bc7ff2a 100644 --- a/src/java/org/apache/cassandra/dht/Range.java +++ b/src/java/org/apache/cassandra/dht/Range.java @@ -201,6 +201,38 @@ public Set> intersectionWith(Range that) return intersectionOneWrapping(that, this); } + /** + * Returns the intersection of this range with the provided one, assuming neither are wrapping. + * + * @param that the other range to return the intersection with. It must not be wrapping. + * @return the intersection of {@code this} and {@code that}, or {@code null} if both ranges don't intersect. + */ + public Range intersectionNonWrapping(Range that) + { + assert !isTrulyWrapAround() && !that.isTrulyWrapAround() : this + " and " + that; + + if (left.compareTo(that.left) < 0) + { + if (right.isMinimum() || (!that.right.isMinimum() && right.compareTo(that.right) >= 0)) + return that; // this contains that. + + if (right.compareTo(that.left) <= 0) + return null; // this is fully before that. + + return new Range<>(that.left, right); + } + else + { + if (that.right.isMinimum() || (!right.isMinimum() && that.right.compareTo(right) >= 0)) + return this; // that contains this. + + if (that.right.compareTo(left) <= 0) + return null; // that is fully before this. + + return new Range<>(left, that.right); + } + } + private static > Set> intersectionBothWrapping(Range first, Range that) { Set> intersection = new HashSet>(2); @@ -456,6 +488,32 @@ public boolean isWrapAround() return isWrapAround(left, right); } + /** + * Checks if the range truly wraps around. + * + * This exists only because {@link #isWrapAround()} is a tad dumb and return true if right is the minimum token, + * no matter what left is, but for most intent and purposes, such range doesn't truly warp around (unwrap produces + * the identity in this case). + *

    + * Also note that it could be that the remaining uses of {@link #isWrapAround()} could be replaced by this method, + * but that is to be checked carefully at some other time (Sylvain). + *

    + * The one thing this method guarantees is that if it's true, then {@link #unwrap()} will return a list with + * exactly 2 ranges, never one. + * + * @return whether the range "true" wraps around. + */ + public boolean isTrulyWrapAround() + { + return isTrulyWrapAround(left, right); + } + + public static > boolean isTrulyWrapAround(T left, T right) + { + T minValue = right.minValue(); + return isWrapAround(left, right) && !right.equals(minValue); + } + /** * @return A copy of the given list of with all ranges unwrapped, sorted by left bound and with overlapping bounds merged. */ diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java index ebf0f0335012..26060769eb14 100644 --- a/src/java/org/apache/cassandra/dht/RangeStreamer.java +++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java @@ -467,12 +467,14 @@ else if (useStrictConsistency) if (useStrictConsistency) { EndpointsForRange strictEndpoints; + + //Start with two sets of who replicates the range before and who replicates it after + EndpointsForRange newEndpoints = strat.calculateNaturalReplicas(toFetch.range().right, tmdAfter); + //Due to CASSANDRA-5953 we can have a higher RF than we have endpoints. //So we need to be careful to only be strict when endpoints == RF - if (oldEndpoints.size() == strat.getReplicationFactor().allReplicas) + if (!oldEndpoints.stream().allMatch(newEndpoints::contains)) { - //Start with two sets of who replicates the range before and who replicates it after - EndpointsForRange newEndpoints = strat.calculateNaturalReplicas(toFetch.range().right, tmdAfter); logger.debug("Old endpoints {}", oldEndpoints); logger.debug("New endpoints {}", newEndpoints); diff --git a/src/java/org/apache/cassandra/dht/Splitter.java b/src/java/org/apache/cassandra/dht/Splitter.java index 857844843461..07709514ab09 100644 --- a/src/java/org/apache/cassandra/dht/Splitter.java +++ b/src/java/org/apache/cassandra/dht/Splitter.java @@ -118,10 +118,75 @@ public double positionInRange(Token token, Range range) return new BigDecimal(elapsedTokens(token, range)).divide(new BigDecimal(tokensInRange(range)), 3, BigDecimal.ROUND_HALF_EVEN).doubleValue(); } - public List splitOwnedRanges(int parts, List weightedRanges, boolean dontSplitRanges) + /** + * How local ranges should be split + */ + public enum SplitType + { + /** Local ranges should always be split, without attempting to keep them whole */ + ALWAYS_SPLIT, + /** A first pass will try to avoid splitting ranges, but if there aren't enough parts, + * then ranges will be split in a second pass. + */ + PREFER_WHOLE, + /** Ranges Should never be split */ + ONLY_WHOLE + } + + /** + * The result of a split operation, this is just a wrapper of the boundaries and the type + * of split that was done, i.e. if the local ranges were split or not. This is just so that + * we can test the algorithm. + */ + public final static class SplitResult + { + public final List boundaries; + public final boolean rangesWereSplit; + + SplitResult(List boundaries, boolean rangesWereSplit) + { + this.boundaries = boundaries; + this.rangesWereSplit = rangesWereSplit; + } + + @Override + public boolean equals(Object o) + { + if (this == o) + return true; + + if (!(o instanceof SplitResult)) + return false; + + SplitResult splitResult = (SplitResult) o; + return Objects.equals(boundaries, splitResult.boundaries) + && Objects.equals(rangesWereSplit, splitResult.rangesWereSplit); + } + + @Override + public int hashCode() + { + return Objects.hash(boundaries, rangesWereSplit); + } + } + + /** + * Split the local ranges into the specified number of parts. + * + * Depending on the parameter {@code splitType}, it may attempt to only merge the local ranges first, + * to see if this is sufficient to cover the requested number of parts. If it's not, it will then split + * existing ranges. + * + * @param parts the number of parts + * @param weightedRanges the local ranges owned by this node + * @param splitType how local ranges should be split, see {@link SplitType} + * + * @return the split result, which contains a list of tokens, one per part, and if the ranges were split or not + */ + public SplitResult splitOwnedRanges(int parts, List weightedRanges, SplitType splitType) { - if (weightedRanges.isEmpty() || parts == 1) - return Collections.singletonList(partitioner.getMaximumToken()); + if (weightedRanges.isEmpty() || parts <= 1) + return new SplitResult(Collections.singletonList(partitioner.getMaximumToken()), false); BigInteger totalTokens = BigInteger.ZERO; for (WeightedRange weightedRange : weightedRanges) @@ -132,12 +197,23 @@ public List splitOwnedRanges(int parts, List weightedRange BigInteger perPart = totalTokens.divide(BigInteger.valueOf(parts)); // the range owned is so tiny we can't split it: if (perPart.equals(BigInteger.ZERO)) - return Collections.singletonList(partitioner.getMaximumToken()); - - if (dontSplitRanges) - return splitOwnedRangesNoPartialRanges(weightedRanges, perPart, parts); + return new SplitResult(Collections.singletonList(partitioner.getMaximumToken()), false); List boundaries = new ArrayList<>(); + + if (splitType != SplitType.ALWAYS_SPLIT) + { + // see if we can obtain a sufficient number of parts by only merging local ranges + boundaries = splitOwnedRangesNoPartialRanges(weightedRanges, perPart, parts); + // we were either able to obtain sufficient parts without splitting ranges or we should never split ranges + if (splitType == SplitType.ONLY_WHOLE || boundaries.size() == parts) + return new SplitResult(boundaries, false); + else + boundaries.clear(); + } + + // otherwise continue by splitting ranges + BigInteger sum = BigInteger.ZERO; for (WeightedRange weightedRange : weightedRanges) { @@ -156,7 +232,7 @@ public List splitOwnedRanges(int parts, List weightedRange boundaries.set(boundaries.size() - 1, partitioner.getMaximumToken()); assert boundaries.size() == parts : boundaries.size() + "!=" + parts + " " + boundaries + ":" + weightedRanges; - return boundaries; + return new SplitResult(boundaries, true); } private List splitOwnedRangesNoPartialRanges(List weightedRanges, BigInteger perPart, int parts) @@ -290,6 +366,11 @@ public Range range() return range; } + public double weight() + { + return weight; + } + public String toString() { return "WeightedRange{" + diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java index d8e82f82c510..5af8d7609a41 100644 --- a/src/java/org/apache/cassandra/dht/Token.java +++ b/src/java/org/apache/cassandra/dht/Token.java @@ -26,6 +26,8 @@ import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; public abstract class Token implements RingPosition, Serializable { @@ -37,8 +39,38 @@ public static abstract class TokenFactory { public abstract ByteBuffer toByteArray(Token token); public abstract Token fromByteArray(ByteBuffer bytes); + + /** + * Produce a weakly prefix-free byte-comparable representation of the token, i.e. such a sequence of bytes that any + * pair x, y of valid tokens of this type and any bytes b1, b2 between 0x10 and 0xEF, + * (+ stands for concatenation) + * compare(x, y) == compareLexicographicallyUnsigned(asByteComparable(x)+b1, asByteComparable(y)+b2) + * (i.e. the values compare like the original type, and an added 0x10-0xEF byte at the end does not change that) and: + * asByteComparable(x)+b1 is not a prefix of asByteComparable(y) (weakly prefix free) + * (i.e. a valid representation of a value may be a prefix of another valid representation of a value only if the + * following byte in the latter is smaller than 0x10 or larger than 0xEF). These properties are trivially true if + * the encoding compares correctly and is prefix free, but also permits a little more freedom that enables somewhat + * more efficient encoding of arbitrary-length byte-comparable blobs. + */ + public ByteSource asComparableBytes(Token token, ByteComparable.Version version) + { + return token.asComparableBytes(version); + } + + /** + * Translates the given byte-comparable representation to a token instance. If the given bytes don't correspond + * to the encoding of an instance of the expected token type, an {@link IllegalArgumentException} may be thrown. + * + * @param comparableBytes A byte-comparable representation (presumably of a token of some expected token type). + * @return A new {@link Token} instance, corresponding to the given byte-ordered representation. If we were + * to call {@link #asComparableBytes(ByteComparable.Version)} on the returned object, we should get a + * {@link ByteSource} equal to the input one as a result. + */ + public abstract Token fromComparableBytes(ByteSource.Peekable comparableBytes, ByteComparable.Version version); + public abstract String toString(Token token); // serialize as string, not necessarily human-readable public abstract Token fromString(String string); // deserialize + public abstract void validate(String token) throws ConfigurationException; public void serialize(Token token, DataOutputPlus out) throws IOException @@ -99,6 +131,34 @@ public long serializedSize(Token object, int version) abstract public long getHeapSize(); abstract public Object getTokenValue(); + /** + * This methods exists so that callers can access the primitive {@code long} value for this {@link Token}, if + * one exits. It is especially useful when the auto-boxing induced by a call to {@link #getTokenValue()} would + * be unacceptable for reasons of performance. + * + * @return the primitive {@code long} value of this token, if one exists + * + * @throws UnsupportedOperationException if this {@link Token} is not backed by a primitive {@code long} value + */ + public long getLongValue() + { + throw new UnsupportedOperationException(); + } + + /** + * Produce a weakly prefix-free byte-comparable representation of the token, i.e. such a sequence of bytes that any + * pair x, y of valid tokens of this type and any bytes b1, b2 between 0x10 and 0xEF, + * (+ stands for concatenation) + * compare(x, y) == compareLexicographicallyUnsigned(asByteComparable(x)+b1, asByteComparable(y)+b2) + * (i.e. the values compare like the original type, and an added 0x10-0xEF byte at the end does not change that) and: + * asByteComparable(x)+b1 is not a prefix of asByteComparable(y) (weakly prefix free) + * (i.e. a valid representation of a value may be a prefix of another valid representation of a value only if the + * following byte in the latter is smaller than 0x10 or larger than 0xEF). These properties are trivially true if + * the encoding compares correctly and is prefix free, but also permits a little more freedom that enables somewhat + * more efficient encoding of arbitrary-length byte-comparable blobs. + */ + abstract public ByteSource asComparableBytes(ByteComparable.Version version); + /** * Returns a measure for the token space covered between this token and next. * Used by the token allocation algorithm (see CASSANDRA-7032). @@ -128,7 +188,7 @@ public boolean isMinimum() /* * A token corresponds to the range of all the keys having this token. - * A token is thus no comparable directly to a key. But to be able to select + * A token is thus not comparable directly to a key. But to be able to select * keys given tokens, we introduce two "fake" keys for each token T: * - lowerBoundKey: a "fake" key representing the lower bound T represents. * In other words, lowerBoundKey is the smallest key that @@ -190,6 +250,13 @@ public int compareTo(PartitionPosition pos) return ((pos instanceof KeyBound) && !((KeyBound)pos).isMinimumBound) ? 0 : 1; } + @Override + public ByteSource asComparableBytes(Version version) + { + int terminator = isMinimumBound ? ByteSource.LT_NEXT_COMPONENT : ByteSource.GT_NEXT_COMPONENT; + return ByteSource.withTerminator(terminator, token.asComparableBytes(version)); + } + public IPartitioner getPartitioner() { return getToken().getPartitioner(); diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java index 1cdbdb544d28..c32dfc700dc1 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java @@ -35,7 +35,8 @@ public enum RequestFailureReason UNKNOWN (0), READ_TOO_MANY_TOMBSTONES (1), TIMEOUT (2), - INCOMPATIBLE_SCHEMA (3); + INCOMPATIBLE_SCHEMA (3), + INDEX_NOT_AVAILABLE (4); public static final Serializer serializer = new Serializer(); @@ -46,6 +47,13 @@ public enum RequestFailureReason this.code = code; } + public int codeForNativeProtocol() + { + // We explicitly indicated in the protocol spec that drivers should not error out on unknown code, and we + // currently support a superset of the OSS codes, so we don't yet worry about the version. + return code; + } + private static final RequestFailureReason[] codeToReasonMap; static diff --git a/src/java/org/apache/cassandra/gms/ApplicationState.java b/src/java/org/apache/cassandra/gms/ApplicationState.java index 4e20d6204884..33a5632b0330 100644 --- a/src/java/org/apache/cassandra/gms/ApplicationState.java +++ b/src/java/org/apache/cassandra/gms/ApplicationState.java @@ -47,6 +47,7 @@ public enum ApplicationState INTERNAL_ADDRESS_AND_PORT, //Replacement for INTERNAL_IP with up to two ports NATIVE_ADDRESS_AND_PORT, //Replacement for RPC_ADDRESS STATUS_WITH_PORT, //Replacement for STATUS + DISK_USAGE, /** * The set of sstable versions on this node. This will usually be only the "current" sstable format (the one with * which new sstables are written), but may contain more on newly upgraded nodes before `upgradesstable` has been @@ -56,6 +57,7 @@ public enum ApplicationState * a comma-separated list. **/ SSTABLE_VERSIONS, + INDEX_STATUS, // DO NOT EDIT OR REMOVE PADDING STATES BELOW - only add new states above. See CASSANDRA-16484 X1, X2, diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 39fe1fe7287c..5dd121831008 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -762,7 +762,6 @@ public void assassinateEndpoint(String address) throws UnknownHostException InetAddressAndPort endpoint = InetAddressAndPort.getByName(address); runInGossipStageBlocking(() -> { EndpointState epState = endpointStateMap.get(endpoint); - Collection tokens; logger.warn("Assassinating {} via gossip", endpoint); if (epState == null) @@ -787,6 +786,7 @@ else if (newState.getHeartBeatState().getHeartBeatVersion() != heartbeat) epState.getHeartBeatState().forceNewerGenerationUnsafe(); } + Collection tokens = null; try { tokens = StorageService.instance.getTokenMetadata().getTokens(endpoint); @@ -794,8 +794,10 @@ else if (newState.getHeartBeatState().getHeartBeatVersion() != heartbeat) catch (Throwable th) { JVMStabilityInspector.inspectThrowable(th); - // TODO this is broken - logger.warn("Unable to calculate tokens for {}. Will use a random one", address); + } + if (tokens == null || tokens.isEmpty()) + { + logger.warn("Trying to assassinate an endpoint {} that does not have any tokens assigned. This should not have happened, trying to continue with a random token.", address); tokens = Collections.singletonList(StorageService.instance.getTokenMetadata().partitioner.getRandomToken()); } @@ -1014,7 +1016,7 @@ void doStatusCheck() // to make sure that the previous read data was correct logger.info("Race condition marking {} as a FatClient; ignoring", endpoint); return; - } + } removeEndpoint(endpoint); // will put it in justRemovedEndpoints to respect quarantine delay evictFromMembership(endpoint); // can get rid of the state immediately }); diff --git a/src/java/org/apache/cassandra/gms/VersionedValue.java b/src/java/org/apache/cassandra/gms/VersionedValue.java index 880cb98e067f..532ced35f403 100644 --- a/src/java/org/apache/cassandra/gms/VersionedValue.java +++ b/src/java/org/apache/cassandra/gms/VersionedValue.java @@ -172,6 +172,11 @@ public VersionedValue load(double load) return new VersionedValue(String.valueOf(load)); } + public VersionedValue diskUsage(String state) + { + return new VersionedValue(state); + } + public VersionedValue schema(UUID newVersion) { return new VersionedValue(newVersion.toString()); @@ -245,6 +250,11 @@ public VersionedValue shutdown(boolean value) return new VersionedValue(VersionedValue.SHUTDOWN + VersionedValue.DELIMITER + value); } + public VersionedValue indexStatus(String status) + { + return new VersionedValue(status); + } + public VersionedValue datacenter(String dcId) { return new VersionedValue(dcId); diff --git a/src/java/org/apache/cassandra/guardrails/Guardrail.java b/src/java/org/apache/cassandra/guardrails/Guardrail.java new file mode 100644 index 000000000000..795237490861 --- /dev/null +++ b/src/java/org/apache/cassandra/guardrails/Guardrail.java @@ -0,0 +1,851 @@ +/* + * 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.guardrails; + +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.LongSupplier; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.service.ClientWarn; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.NoSpamLogger; +import org.apache.cassandra.utils.units.SizeUnit; +import org.apache.cassandra.utils.units.Units; + +import static java.lang.String.format; + +/** + * General class defining a given guardrail (that guards against some particular usage/condition). + * + *

    Some guardrails only emit warnings when triggered, while other fail the query that trigger them. Some may do one + * or the other based on specific threshold. + * + *

    Note that all the defined class support live updates, which is why each guardrail class ctor takes suppliers of + * the condition the guardrail acts on rather than the condition itself. Which does imply that said suppliers should + * be fast and non-blocking to avoid surprises. Note that this does not mean live updates are exposed to the user, + * just that the implementation is up to it if we ever want to expose it. + */ +public abstract class Guardrail +{ + private static final NoSpamLogger logger = NoSpamLogger.getLogger(LoggerFactory.getLogger(Guardrail.class), + 10, TimeUnit.MINUTES); + private static final String REDACTED = ""; + + /** A name identifying the guardrail (mainly for shipping with Insights events). */ + public final String name; + + /** whether to throw {@link InvalidRequestException} on {@link this#fail(String)} */ + private boolean throwOnFailure = true; + + /** minimum logging and triggering interval to avoid spamming downstream*/ + private long minNotifyIntervalInMs = 0; + + /** time of last warning in milliseconds */ + private volatile long lastWarnInMs = 0; + + /** time of last failure in milliseconds */ + private volatile long lastFailInMs = 0; + + Guardrail(String name) + { + this.name = name; + } + + protected void warn(String fullMessage, String redactedMessage) + { + if (skipNotifying(true)) + return; + + logger.warn(fullMessage); + // Note that ClientWarn will simply ignore the message if we're not running this as part of a user query + // (the internal "state" will be null) + ClientWarn.instance.warn(fullMessage); + // Similarly, tracing will also ignore the message if we're not running tracing on the current thread. + Tracing.trace(fullMessage); + for (Guardrails.Listener listener : Guardrails.listeners) + listener.onWarningTriggered(name, redactedMessage); + } + + protected void warn(String fullMessage) + { + warn(fullMessage, fullMessage); + } + + protected void fail(String fullMessage, String redactedMessage) + { + if (!skipNotifying(false)) + { + logger.error(fullMessage); + // Tracing will ignore the message if we're not running tracing on the current thread. + Tracing.trace(fullMessage); + for (Guardrails.Listener listener : Guardrails.listeners) + listener.onFailureTriggered(name, redactedMessage); + } + + if (throwOnFailure) + throw new InvalidRequestException(fullMessage); + } + + protected void fail(String message) + { + fail(message, message); + } + + /** + * do no throw {@link InvalidRequestException} if guardrail failure is triggered. + * + * Note: this method is not thread safe and should only be used during guardrail initialization + * + * @return current guardrail + */ + Guardrail noExceptionOnFailure() + { + this.throwOnFailure = false; + return this; + } + + /** + * Note: this method is not thread safe and should only be used during guardrail initialization + * + * @param minNotifyIntervalInMs frequency of logging and triggering listener to avoid spamming, + * default 0 means always log and trigger listeners. + * @return current guardrail + */ + Guardrail minNotifyIntervalInMs(long minNotifyIntervalInMs) + { + assert minNotifyIntervalInMs >= 0; + this.minNotifyIntervalInMs = minNotifyIntervalInMs; + return this; + } + + /** + * reset last notify time to make sure it will notify downstream when {@link this#warn(String, String)} + * or {@link this#fail(String)} is called next time. + */ + @VisibleForTesting + public void resetLastNotifyTime() + { + lastFailInMs = 0; + lastWarnInMs = 0; + } + + /** + * @return true if guardrail should not log message and trigger listeners; otherwise, update lastWarnInMs or + * lastFailInMs respectively. + */ + private boolean skipNotifying(boolean isWarn) + { + if (minNotifyIntervalInMs == 0) + return false; + + long nowInMs = System.currentTimeMillis(); + long timeElapsedInMs = nowInMs - (isWarn ? lastWarnInMs : lastFailInMs); + + boolean skip = timeElapsedInMs < minNotifyIntervalInMs; + + if (!skip) + { + if (isWarn) + lastWarnInMs = nowInMs; + else + lastFailInMs = nowInMs; + } + + return skip; + } + + /** + * Checks whether this guardrail is enabled or not. This will be enabled if guardrails are globally enabled and + * {@link Guardrails#ready()} and if the authenticated user (if specified) is not a system nor superuser. + * + * @param queryState the queryState, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + * @return {@code true} if this guardrail is enabled & ready, {@code false} otherwise. + */ + public boolean enabled(@Nullable QueryState queryState) + { + return Guardrails.ready() && (queryState == null || queryState.isOrdinaryUser()); + } + + /** + * A guardrail based on numeric threshold(s). + * + *

    A {@link Threshold} guardrail defines (up to) 2 threshold, one at which a warning is issued, and a higher one + * at which a failure is triggered. Only one of those thresholds can be activated if desired. + * + *

    This guardrail only handles guarding positive values. + */ + public static class Threshold extends Guardrail + { + /** + * A {@link Threshold} with both failure and warning thresholds disabled, so that cannot ever be triggered. + */ + public static final Threshold NEVER_TRIGGERED = new Threshold("never_triggered", () -> -1L, () -> -1L, null); + + /** + * A function used to build the error message of a triggered {@link Threshold} guardrail. + */ + public interface ErrorMessageProvider + { + /** + * Called when the guardrail is triggered to build the corresponding error message. + * + * @param isWarning whether the trigger is a warning one; otherwise it is failure one. + * @param what a string, provided by the call to the {@link #guard} method, describing what the guardrail + * has been applied to (and that has triggered it). + * @param valueString the value that triggered the guardrail (as a string). + * @param thresholdString the threshold that was passed to trigger the guardrail (as a string). + */ + String createMessage(boolean isWarning, String what, String valueString, String thresholdString); + } + + final LongSupplier warnThreshold; + final LongSupplier failThreshold; + final ErrorMessageProvider errorMessageProvider; + + /** + * Creates a new {@link Threshold} guardrail. + * + * @param name the name of the guardrail (for identification in {@link Guardrails.Listener} events). + * @param warnThreshold a supplier of the threshold above which a warning should be triggered. This cannot be + * null, but {@code () -> -1L} can be provided if no warning threshold is desired. + * @param failThreshold a supplier of the threshold above which a failure should be triggered. This cannot be + * null, but {@code () -> -1L} can be provided if no failure threshold is desired. + * @param errorMessageProvider a function to generate the error message if the guardrail is triggered + * (being it for a warning or a failure). + */ + Threshold(String name, + LongSupplier warnThreshold, + LongSupplier failThreshold, + ErrorMessageProvider errorMessageProvider) + { + super(name); + this.warnThreshold = warnThreshold; + this.failThreshold = failThreshold; + this.errorMessageProvider = errorMessageProvider; + } + + protected String errMsg(boolean isWarning, String what, long value, long thresholdValue) + { + return errorMessageProvider.createMessage(isWarning, + what, + Long.toString(value), + Long.toString(thresholdValue)); + } + + protected String redactedErrMsg(boolean isWarning, long value, long thresholdValue) + { + return errorMessageProvider.createMessage(isWarning, + REDACTED, + Long.toString(value), + Long.toString(thresholdValue)); + } + + private long failValue() + { + long failValue = failThreshold.getAsLong(); + return failValue < 0 ? Long.MAX_VALUE : failValue; + } + + private long warnValue() + { + long warnValue = warnThreshold.getAsLong(); + return warnValue < 0 ? Long.MAX_VALUE : warnValue; + } + + /** + * Checks whether this guardrail is enabled or not. This will be enabled if guardrails are + * ({@link Guardrails#ready()} ()}), the keyspace (if specified) is not an internal one, and if any of the + * thresholds is positive. + * + * @param queryState the queryState, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + * @return {@code true} if this guardrail is enabled, {@code false} otherwise. + */ + public boolean enabled(@Nullable QueryState queryState) + { + return super.enabled(queryState) && (failThreshold.getAsLong() >= 0 || warnThreshold.getAsLong() >= 0); + } + + /** + * Checks whether the provided value would trigger a warning or failure if passed to {@link #guard}. + * + *

    This method is optional (does not have to be called) but can be used in the case where the "what" + * argument to {@link #guard} is expensive to build to save doing so in the common case (of the guardrail + * not being triggered). + * + * @param value the value to test. + * @param queryState the queryState, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + * @return {@code true} if {@code value} is above the warning or failure thresholds of this guardrail, + * {@code false otherwise}. + */ + public boolean triggersOn(long value, @Nullable QueryState queryState) + { + return enabled(queryState) && (value > Math.min(failValue(), warnValue())); + } + + /** + * Apply the guardrail to the provided value, triggering a warning or failure if appropriate. + * + * @param value the value to check. + * @param what a string describing what {@code value} is a value of used in the error message if the + * guardrail is triggered (for instance, say the guardrail guards the size of column values, then this + * argument must describe which column of which row is triggering the guardrail for convenience). Note that + * this is only used if the guardrail triggers, so if it is expensive to build, you can put the call to + * this method behind a {@link #triggersOn} call. + * @param containsUserData a boolean describing if {@code what} contains user data. If this is the case, + * {@code what} will only be included in the log messages and client warning. It will not be included in the + * error messages that are passed to listeners and exceptions. We have to exclude the user data from + * exceptions because they are sent to Insights. + * @param queryState the query state, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + */ + public void guard(long value, String what, boolean containsUserData, @Nullable QueryState queryState) + { + if (!enabled(queryState)) + return; + + long failValue = failValue(); + if (value > failValue) + { + triggerFail(value, failValue, what, containsUserData); + return; + } + + long warnValue = warnValue(); + if (value > warnValue) + triggerWarn(value, warnValue, what, containsUserData); + } + + private void triggerFail(long value, long failValue, String what, boolean containsUserData) + { + String fullMsg = errMsg(false, what, value, failValue); + fail(fullMsg, containsUserData ? redactedErrMsg(false, value, failValue) : fullMsg); + } + + private void triggerWarn(long value, long warnValue, String what, boolean containsUserData) + { + String fullMsg = errMsg(true, what, value, warnValue); + warn(fullMsg, containsUserData ? redactedErrMsg(true, value, warnValue) : fullMsg); + } + + /** + * Creates a new {@link GuardedCounter} guarded by this threshold guardrail. + * + * @param whatFct a function called when either a warning or failure is triggered by the created counter to + * describe the value. This is equivalent to the {@code what} argument of {@link #guard} but is a function to + * allow the output string to be compute lazily (only if a failure/warn ends up being triggered). + * @param containsUserData if a warning or failure is triggered by the created counter and the {@code whatFct} + * is called, indicates whether the create string contains user data. This is the exact equivalent to the + * similarly named argument of {@link #guard}. + * @param queryState the query state, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + * @return the newly created guarded counter. + */ + public GuardedCounter newCounter(Supplier whatFct, boolean containsUserData, @Nullable QueryState queryState) + { + Threshold threshold = enabled(queryState) ? this : NEVER_TRIGGERED; + return threshold.new GuardedCounter(whatFct, containsUserData); + } + + /** + * A facility for when the value to guard is built incrementally, but we want to trigger failures as soon + * as the failure threshold is reached, but only trigger the warning on the final value (and so only if the + * failure threshold hasn't also been reached). + *

    + * Note that instances are neither thread safe nor reusable. + */ + public class GuardedCounter + { + private final long warnValue; + private final long failValue; + private final Supplier what; + private final boolean containsUserData; + + private long accumulated; + + private GuardedCounter(Supplier what, boolean containsUserData) + { + // We capture the warn and fail value at the time of the counter construction to ensure we use + // stable value during the counter lifetime (and reading a final field is possibly at tad faster). + this.warnValue = warnValue(); + this.failValue = failValue(); + this.what = what; + this.containsUserData = containsUserData; + } + + /** + * The currently accumulated value of the counter. + */ + public long get() + { + return accumulated; + } + + /** + * Add the provided increment to the counter, triggering a failure if the counter after this addition + * crosses the failure threshold. + * + * @param increment the increment to add. + */ + public void add(long increment) + { + accumulated += increment; + if (accumulated > failValue) + triggerFail(accumulated, failValue, what.get(), containsUserData); + } + + /** + * Trigger the warn if the currently accumulated counter value crosses warning threshold and the failure + * has not been triggered yet. + *

    + * This is generally meant to be called when the guarded value is complete. + * + * @return {@code true} and trigger a warning if the current counter value is greater than the warning + * threshold and less than or equal to the failure threshold, {@code false} otherwise. + */ + public boolean checkAndTriggerWarning() + { + if (accumulated > warnValue && accumulated <= failValue) + { + triggerWarn(accumulated, warnValue, what.get(), containsUserData); + return true; + } + return false; + } + } + } + + /** + * A {@link Threshold} guardrail whose values represent a byte size. + * + *

    This works exactly as a {@link Threshold}, but provides slightly more convenient error messages (display + * the sizes in human readable format). + */ + public static class SizeThreshold extends Threshold + { + SizeThreshold(String name, + LongSupplier warnThreshold, + LongSupplier failThreshold, + ErrorMessageProvider errorMessageProvider) + { + super(name, warnThreshold, failThreshold, errorMessageProvider); + } + + @Override + protected String errMsg(boolean isWarning, String what, long value, long thresholdValue) + { + return errorMessageProvider.createMessage(isWarning, + what, + Units.toString(value, SizeUnit.BYTES), + Units.toString(thresholdValue, SizeUnit.BYTES)); + } + + @Override + protected String redactedErrMsg(boolean isWarning, long value, long thresholdValue) + { + return errorMessageProvider.createMessage(isWarning, + REDACTED, + Units.toString(value, SizeUnit.BYTES), + Units.toString(thresholdValue, SizeUnit.BYTES)); + } + } + + /** + * A {@link Threshold} guardrail whose values represent a percentage + * + *

    This work exactly as a {@link Threshold}, but provides slightly more convenient error messages for percentage + */ + public static class PercentageThreshold extends Threshold + { + PercentageThreshold(String name, + LongSupplier warnThreshold, + LongSupplier failThreshold, + ErrorMessageProvider errorMessageProvider) + { + super(name, warnThreshold, failThreshold, errorMessageProvider); + } + + @Override + protected String errMsg(boolean isWarning, String what, long value, long thresholdValue) + { + return errorMessageProvider.createMessage(isWarning, + what, + String.format("%d%%", value), + String.format("%d%%", thresholdValue)); + } + } + + /** + * A guardrail that completely disables the use of a particular feature. + * + *

    Note that this guardrail only triggers failures (if the feature is disabled) so is only meant for + * query-based guardrails (we're happy to reject queries deemed dangerous, but we don't want to create a guardrail + * that breaks compaction for instance). + */ + public static class DisableFlag extends Guardrail + { + private final BooleanSupplier disabled; + private final String what; + + /** + * Creates a new {@link DisableFlag} guardrail. + * + * @param name the name of the guardrail (for identification in {@link Guardrails.Listener} events). + * @param disabled a supplier of boolean indicating whether the feature guarded by this guardrail must be + * disabled. + * @param what the feature that is guarded by this guardrail (for reporting in error messages), + * {@link #ensureEnabled(String, QueryState)}}} can specify a different {@code what}. + */ + DisableFlag(String name, BooleanSupplier disabled, String what) + { + super(name); + this.disabled = disabled; + this.what = what; + } + + /** + * Triggers a failure if this guardrail is disabled. + * + *

    This must be called when the feature guarded by this guardrail is used to ensure such use is in fact + * allowed. + * + *

    This must be called when the feature guarded by this guardrail is used to ensure such use is in fact + * allowed. + * + * @param queryState the queryState, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + */ + public void ensureEnabled(@Nullable QueryState queryState) + { + ensureEnabled(what, queryState); + } + + /** + * Triggers a failure if this guardrail is disabled. + * + *

    This must be called when the feature guarded by this guardrail is used to ensure such use is in fact + * allowed. + * + * @param what the feature that is guarded by this guardrail (for reporting in error messages). + * @param queryState the queryState, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + */ + public void ensureEnabled(String what, @Nullable QueryState queryState) + { + if (enabled(queryState) && disabled.getAsBoolean()) + fail(what + " is not allowed"); + } + } + + /** + * Base class for guardrail that are triggered based on a set of values. + * + * @param the type of the values that trigger the guardrail. + */ + private static abstract class ValuesBaseGuardrail extends Guardrail + { + /* + * Implementation note: as mentioned in the class Javadoc and for consistency with the other Guardrail + * implementation of this class (and to generally avoid surprises), this implementation ensures that live + * changes to the underlying guardrail setting gets reflected. This is the reason for the relative + * "complexity" of this class. + */ + + private final Supplier> rawSupplier; + private final Function parser; + protected final String what; + + private volatile ImmutableSet cachedValues; + private volatile Set cachedRaw; + + protected ValuesBaseGuardrail( + String name, Supplier> disallowedRaw, Function parser, String what) + { + super(name); + this.rawSupplier = disallowedRaw; + this.parser = parser; + this.what = what; + + if (Guardrails.ready()) + ensureUpToDate(); + } + + protected void ensureUpToDate() + { + Set current = rawSupplier.get(); + // Same as below, this shouldn't happen if settings have been properly sanitized, but throw a meaningful + // error if there is a bug. + if (current == null) + throw new RuntimeException(format("Invalid null setting for guardrail on %s. This should not have" + + " happened", what)); + + // Note that this will fail on first call (as we want), as currentRaw will be null but not current + if (current == cachedRaw) + return; + + try + { + // Setting cachedAllowed first so that on a parse failure we leave everything as it previously + // was (not that we'd expect that matter but ...). + cachedValues = current.stream() + .map(parser) + .collect(ImmutableSet.toImmutableSet()); + cachedRaw = current; + } + catch (Exception e) + { + // This catches parsing errors. Hopefully, this shouldn't happen as guardrails settings should have + // been sanitized, but ... + // Also, we catch the exception to add a meaningful error message, but rethrow otherwise: if a + // guardrail has been configured, it's presumably to avoid bad things to go in, so we don't want to + // take the risk of letting it go if there is a misconfiguration. + throw new RuntimeException(format("Error parsing configured setting for guardrail on %s. This " + + "is a bug and should not have happened." + + "The failing setting is %s", what, current), e); + } + } + + protected Set matchingValues(Set values) { + return Sets.intersection(values, cachedValues); + } + + protected String triggerValuesString() + { + return cachedRaw.toString(); + } + + /** + * Checks whether the provided value would trigger this guardrail. + * + *

    This method is optional (does not have to be called) but can be used in the case some of the arguments + * to the actual guardrail method is expensive to build to save doing so in the common case (of the + * guardrail not being triggered). + * + * @param value the value to test. + * @param state the query state, used to skip the check if the query is internal or is done by a superuser. + * @return {@code true} if {@code value} is not allowed by this guardrail, + * {@code false otherwise}. + */ + public boolean triggersOn(T value, @Nullable QueryState state) + { + if (!enabled(state)) + return false; + + ensureUpToDate(); + return cachedValues.contains(value); + } + } + + + /** + * A guardrail that rejects the use of specific values. + * + *

    Note that like {@link DisableFlag}, this guardrail only trigger failures and is thus only for query-based + * guardrails. + * + * @param the type of the values of which certain are disallowed. + */ + public static class DisallowedValues extends ValuesBaseGuardrail + { + /** + * Creates a new {@link DisallowedValues} guardrail. + * + * @param name the name of the guardrail (for identification in {@link Guardrails.Listener} events). + * @param disallowedRaw a supplier of the values that are disallowed in raw (string) form. The set returned by + * this supplier must be immutable (we don't use {@code ImmutableSet} because we + * want to feed values from {@link GuardrailsConfig} directly and having ImmutableSet + * there would currently be annoying (because populated automatically by snakeYaml)). + * @param parser a function to parse the value to disallow from string. + * @param what what represents the value disallowed (for reporting in error messages). + */ + DisallowedValues(String name, Supplier> disallowedRaw, Function parser, String what) + { + super(name, disallowedRaw, parser, what); + } + + /** + * Triggers a failure if the provided value is disallowed by this guardrail. + * + * @param value the value to check. + * @param state the query state, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + */ + public void ensureAllowed(T value, @Nullable QueryState state) + { + if (triggersOn(value, state)) + fail(format("Provided value %s is not allowed for %s (disallowed values are: %s)", + value, what, triggerValuesString())); + } + + /** + * Triggers a failure if any of the provided values is disallowed by this guardrail. + * + * @param values the values to check. + * @param state the query state, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + */ + public void ensureAllowed(Set values, @Nullable QueryState state) + { + if (!enabled(state)) + return; + + ensureUpToDate(); + + Set disallowed = matchingValues(values); + if (!disallowed.isEmpty()) + fail(format("Provided values %s are not allowed for %s (disallowed values are: %s)", + disallowed.stream().sorted().collect(Collectors.toList()), what, triggerValuesString())); + } + } + + /** + * A guardrail based on two predicates. + * + *

    A {@link Predicates} guardrail defines (up to) 2 predicates, one at which a warning is issued, and another one + * at which a failure is triggered. If failure is triggered, warning is skipped. + * + * @param the type of the values to be tested against predicates. + */ + public static class Predicates extends Guardrail + { + private final Predicate warnPredicate; + private final Predicate failurePredicate; + private final MessageProvider messageProvider; + + /** + * A function used to build the warning or error message of a triggered {@link Predicates} guardrail. + */ + public interface MessageProvider + { + /** + * Called when the guardrail is triggered to build the corresponding message. + * + * @param isWarning whether the trigger is a warning one; otherwise it is failure one. + * @param value the value that triggers guardrail. + */ + String createMessage(boolean isWarning, T value); + } + + /** + * Creates a new {@link Predicates} guardrail. + * + * @param name the name of the guardrail (for identification in {@link Guardrails.Listener} events). + * @param warnPredicate a predicate that is used to check if given value should trigger a warning. + * @param failurePredicate a predicate that is used to check if given value should trigger a failure. + * @param messageProvider a function to generate the warning or error message if the guardrail is triggered + */ + Predicates(String name, Predicate warnPredicate, Predicate failurePredicate, MessageProvider messageProvider) + { + super(name); + this.warnPredicate = warnPredicate; + this.failurePredicate = failurePredicate; + this.messageProvider = messageProvider; + } + + /** + * Apply the guardrail to the provided value, triggering a warning or failure if appropriate. + * + * @param value the value to check. + * @param queryState the query queryState, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + */ + public void guard(T value, @Nullable QueryState queryState) + { + if (!enabled(queryState)) + return; + + if (failurePredicate.test(value)) + { + fail(messageProvider.createMessage(false, value)); + } + else if (warnPredicate.test(value)) + { + warn(messageProvider.createMessage(true, value)); + } + } + } + + /** + * A guardrail that warns but ignore some specific values. + * + * @param the type of the values of which certain are ignored. + */ + public static class IgnoredValues extends ValuesBaseGuardrail + { + /** + * Creates a new {@link IgnoredValues} guardrail. + * + * @param name the name of the guardrail (for identification in {@link Guardrails.Listener} events). + * @param ignoredRaw a supplier of the values that are ignored in raw (string) form. The set returned by + * this supplier must be immutable (we don't use {@code ImmutableSet} because we + * want to feed values from {@link GuardrailsConfig} directly and having ImmutableSet + * there would currently be annoying (because populated automatically by snakeYaml)). + * @param parser a function to parse the value to ignore from string. + * @param what what represents the value ignored (for reporting in error messages). + */ + IgnoredValues(String name, Supplier> ignoredRaw, Function parser, String what) + { + super(name, ignoredRaw, parser, what); + } + + /** + * Checks for ignored values by this guardrail and when it found some, log a warning and trigger an action + * to ignore them. + * + * @param values the values to check. + * @param ignoreAction an action called on the subset of {@code values} that should be ignored. This action + * should do whatever is necessary to make sure the value is ignored. + * @param state the query state, used to skip the check if the query is internal or is done by a superuser. + * A {@code null} value means that the check should be done regardless of the query. + */ + public void maybeIgnoreAndWarn(Set values, Consumer ignoreAction, @Nullable QueryState state) + { + if (!enabled(state)) + return; + + ensureUpToDate(); + + Set toIgnore = matchingValues(values); + if (toIgnore.isEmpty()) + return; + + warn(format("Ignoring provided values %s as they are not supported for %s (ignored values are: %s)", + toIgnore.stream().sorted().collect(Collectors.toList()), what, triggerValuesString())); + for (T value : toIgnore) + ignoreAction.accept(value); + } + } +} + diff --git a/src/java/org/apache/cassandra/guardrails/Guardrails.java b/src/java/org/apache/cassandra/guardrails/Guardrails.java new file mode 100644 index 000000000000..e23d90f13aa6 --- /dev/null +++ b/src/java/org/apache/cassandra/guardrails/Guardrails.java @@ -0,0 +1,316 @@ +/* + * 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.guardrails; + +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.guardrails.Guardrail.DisableFlag; +import org.apache.cassandra.guardrails.Guardrail.DisallowedValues; +import org.apache.cassandra.guardrails.Guardrail.IgnoredValues; +import org.apache.cassandra.guardrails.Guardrail.PercentageThreshold; +import org.apache.cassandra.guardrails.Guardrail.Predicates; +import org.apache.cassandra.guardrails.Guardrail.SizeThreshold; +import org.apache.cassandra.guardrails.Guardrail.Threshold; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.service.disk.usage.DiskUsageBroadcaster; + +import static java.lang.String.format; + +/** + * Entry point for Guardrails, storing the defined guardrails and provided a few global methods over them. + */ +public abstract class Guardrails +{ + private static final GuardrailsConfig config = DatabaseDescriptor.getGuardrailsConfig(); + + public static final Threshold tablesLimit = + new Threshold("number_of_tables", + () -> config.tables_warn_threshold, + () -> config.tables_failure_threshold, + (isWarning, what, v, t) -> isWarning + ? format("Creating table %s, current number of tables %s exceeds warning threshold of %s.", + what, v, t) + : format("Cannot have more than %s tables, failed to create table %s", + t, what)); + + public static final DisallowedValues disallowedTableProperties = + new DisallowedValues<>("disallowed_table_properties", + () -> config.table_properties_disallowed, + String::toLowerCase, + "Table Properties"); + + public static final IgnoredValues ignoredTableProperties = + new IgnoredValues<>("ignored_table_properties", + () -> config.table_properties_ignored, + String::toLowerCase, + "Table Properties"); + + public static final DisableFlag counterEnabled = + new DisableFlag("counter", + () -> !config.counter_enabled, + "Counter"); + + public static final Threshold columnValueSize = + new SizeThreshold("column_value_size", + () -> -1L, // not needed so far + () -> config.column_value_size_failure_threshold_in_kb * 1024L, + (x, what, v, t) -> format("Value of %s of size %s is greater than the maximum allowed (%s)", + what, v, t)); + + public static final Threshold columnsPerTable = + new Threshold("columns_per_table", + () -> -1L, // not needed so far + () -> config.columns_per_table_failure_threshold, + (x, what, v, t) -> format("Tables cannot have more than %s columns, but %s provided for table %s", + t, v, what)); + + public static final Threshold fieldsPerUDT = + new Threshold("fields_per_udt", + () -> -1L, // not needed so far + () -> config.fields_per_udt_failure_threshold, + (x, what, v, t) -> format("User types cannot have more than %s columns, but %s provided for type %s", + t, v, what)); + + public static final Threshold collectionSize = + new SizeThreshold("collection_size", + () -> config.collection_size_warn_threshold_in_kb * 1024L, + () -> -1L, // not needed so far + (x, what, v, t) -> format("Detected collection %s of size %s, greater than the maximum recommended size (%s)", + what, v, t)); + + public static final Threshold itemsPerCollection = + new Threshold("items_per_collection", + () -> config.items_per_collection_warn_threshold, + () -> -1L, // not needed so far + (x, what, v, t) -> format("Detected collection %s with %s items, greater than the maximum recommended (%s)", + what, v, t)); + + public static final DisableFlag readBeforeWriteListOperationsEnabled = + new DisableFlag("read_before_write_list_operations", + () -> !config.read_before_write_list_operations_enabled, + "List operation requiring read before write"); + + public static final DisableFlag userTimestampsEnabled = + new DisableFlag("user_provided_timestamps", + () -> !config.user_timestamps_enabled, + "User provided timestamps (USING TIMESTAMP)"); + + public static final DisableFlag loggedBatchEnabled = + new DisableFlag("logged_batch", + () -> !config.logged_batch_enabled, + "LOGGED batch"); + + public static final DisableFlag truncateTableEnabled = + new DisableFlag("truncate_table", + () -> !config.truncate_table_enabled, + "TRUNCATE table"); + + public static final DisallowedValues disallowedWriteConsistencies = + new DisallowedValues<>("disallowed_write_consistency_levels", + () -> config.write_consistency_levels_disallowed, + ConsistencyLevel::fromString, + "Write Consistency Level"); + + public static final Threshold secondaryIndexesPerTable = + new Threshold("secondary_indexes_per_table", + () -> -1, + () -> config.secondary_index_per_table_failure_threshold, + (x, what, v, t) -> format("Tables cannot have more than %s secondary indexes, failed to create secondary index %s", + t, what)); + + public static final Threshold indexesPerTableSasi = + new Threshold("sasi_indexes_per_table_failure_threshold", + () -> -1, + () -> config.sasi_indexes_per_table_failure_threshold, + (x, what, v, t) -> format("Tables cannot have more than %s SASI indexes, failed to create SASI index %s", + t, what)); + + public static final Threshold indexesPerTableSai = + new Threshold("sai_indexes_per_table_failure_threshold", + () -> -1, + () -> config.sai_indexes_per_table_failure_threshold, + (x, what, v, t) -> format("Tables cannot have more than %s StorageAttachedIndex secondary indexes, failed to create secondary index %s", + t, what)); + + public static final Threshold indexesTotalSai = + new Threshold("sai_indexes_total_failure_threshold", + () -> -1, + () -> config.sai_indexes_total_failure_threshold, + (x, what, v, t) -> format("Cannot have more than %s StorageAttachedIndex secondary indexes across all keyspaces, failed to create secondary index %s", + t, what)); + + public static final Threshold materializedViewsPerTable = + new Threshold("materialized_views_per_table", + () -> -1, + () -> config.materialized_view_per_table_failure_threshold, + (x, what, v, t) -> format("Tables cannot have more than %s materialized views, failed to create materialized view %s", + t, what)); + + public static final Threshold pageSize = + new SizeThreshold("page_size", + () -> -1L, + () -> config.page_size_failure_threshold_in_kb * 1024L, + (x, what, v, t) -> format("Page size %s - %s is greater than the maximum allowed (%s)", + what, v, t)); + + public static final Threshold partitionSize = + new SizeThreshold("partition_size", + () -> config.partition_size_warn_threshold_in_mb * 1024L * 1024L, + () -> -1L, + (x, what, v, t) -> format("Detected partition %s of size %s is greater than the maximum recommended size (%s)", + what, v, t)); + + public static final Threshold partitionKeysInSelectQuery = + new Threshold("partition_keys_in_select_query", + () -> -1L, + () -> config.partition_keys_in_select_failure_threshold, + (x, what, v, t) -> format("%s cannot be completed because it selects %s partitions keys - more than the maximum allowed %s", what, v, t)); + + public static final Threshold inSelectCartesianProduct = + new Threshold("in_select_cartesian_product", + () -> -1L, + () -> config.in_select_cartesian_product_failure_threshold, + (x, what, v, t) -> format("The query cannot be completed because cartesian product of all values in IN conditions is greater than %s", t)); + + @SuppressWarnings("unchecked") + public static final Predicates replicaDiskUsage = + (Predicates) new Predicates<>("replica_disk_usage", + DiskUsageBroadcaster.instance::isStuffed, + DiskUsageBroadcaster.instance::isFull, + // not using `what` because it represents replica address which should be hidden from client. + (isWarning, what) -> isWarning + ? "Replica disk usage exceeds warn threshold" + : "Write request failed because disk usage exceeds failure threshold") + .minNotifyIntervalInMs(TimeUnit.MINUTES.toMillis(30)); + + public static final PercentageThreshold localDiskUsage = + (PercentageThreshold) new PercentageThreshold("local_disk_usage", + () -> config.disk_usage_percentage_warn_threshold, + () -> config.disk_usage_percentage_failure_threshold, + (isWarning, what, v, t) -> isWarning + ? format("Local disk usage %s(%s) exceeds warn threshold of %s", v, what, t) + : format("Local disk usage %s(%s) exceeds failure threshold of %s, will stop accepting writes", v, what, t)) + .noExceptionOnFailure() + .minNotifyIntervalInMs(TimeUnit.MINUTES.toMillis(30)); + + public static final Threshold scannedTombstones = + new Threshold("scanned_tombstones", + () -> config.tombstone_warn_threshold, + () -> config.tombstone_failure_threshold, + (isWarning, what, v, t) -> isWarning ? + format("Scanned over %s tombstone rows for query %1.512s - more than the warning threshold %s", v, what, t) : + format("Scanned over %s tombstone rows during query %1.512s - more than the maximum allowed %s; query aborted", v, what, t)); + + + public static final Threshold batchSize = + new SizeThreshold("batch_size", + config::getBatchSizeWarnThreshold, + config::getBatchSizeFailThreshold, + (isWarning, what, v, t) -> isWarning + ? format("Batch for %s is of size %s, exceeding specified warning threshold %s", what, v, t) + : format("Batch for %s is of size %s, exceeding specified failure threshold %s", what, v, t)); + + public static final Threshold unloggedBatchAcrossPartitions = + new Threshold("unlogged_batch_across_partitions", + () -> config.unlogged_batch_across_partitions_warn_threshold, + () -> -1L, + (x, what, v, t) -> format("Unlogged batch covering %s partitions detected " + + "against table%s %s. You should use a logged batch for " + + "atomicity, or asynchronous writes for performance.", + v, what.contains(", ") ? "s" : "", what)); + + static final List listeners = new CopyOnWriteArrayList<>(); + + private Guardrails() + {} + + /** + * Whether guardrails are ready. + * + * @return {@code true} if daemon is initialized (applies based on their individual setting), {@code false} + * otherwise (in which case no guardrail will trigger). + */ + public static boolean ready() + { + return DatabaseDescriptor.isDaemonInitialized(); + } + + /** + * Register a {@link Listener}. + * + *

    Note that listeners are called in the order they are registered, and on the thread on which the guardrail + * is triggered. + * + * @param listener the listener to register. If the same listener is registered twice (or more), its method will be + * called twice (or more) for every trigger. + */ + public static void register(Listener listener) + { + listeners.add(listener); + } + + /** + * Unregister a previously registered listener. + * + * @param listener the listener to unregister. If it was not registered before, this is a no-op. If it was + * registered more than once, only one of the instance is unregistered. + */ + public static void unregister(Listener listener) + { + listeners.remove(listener); + } + + /** + * Interface for external listener interested in being notified when a guardrail is triggered. + * + *

    Listeners should be registered through the {@link #register} method to take effect. + * + *

    Note: this provides a mechanism to generate events when guardrails are triggered. + */ + public interface Listener + { + /** + * Called when a guardrail triggers a warning. + * + *

    This method is called on the thread on which the guardrail is triggered. + * Overall, if any blocking work is to be done, the method should submit it asynchronously on a + * separate dedicated thread. + * + * @param guardrailName a name describing the guardrail. + * @param message the message corresponding to the guardrail trigger. + */ + public void onWarningTriggered(String guardrailName, String message); + + /** + * Called when a guardrail triggers a failure. + * + *

    This method is called on the thread on which the guardrail is triggered. + * Overall, if any blocking work is to be done, the method should submit it asynchronously on a + * separate dedicated thread. + * + * @param guardrailName a name describing the guardrail. + * @param message the message corresponding to the guardrail trigger. + */ + public void onFailureTriggered(String guardrailName, String message); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/guardrails/GuardrailsConfig.java b/src/java/org/apache/cassandra/guardrails/GuardrailsConfig.java new file mode 100644 index 000000000000..d5c14983d0d6 --- /dev/null +++ b/src/java/org/apache/cassandra/guardrails/GuardrailsConfig.java @@ -0,0 +1,448 @@ +/* + * 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.guardrails; + +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; + +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.statements.schema.TableAttributes; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.utils.units.SizeUnit; + +import static java.lang.String.format; + +/** + * Configuration settings for guardrails (populated from the Yaml file). + * + *

    Note that the settings here must only be used by the {@link Guardrails} class and not directly by the code + * checking each guarded constraint (which, again, should use the higher level abstractions defined in + * {@link Guardrails}). + * + *

    We have 2 variants of guardrails, soft (warn) and hard (fail) limits, each guardrail having either one of the + * variant or both (note in particular that hard limits only make sense for guardrails triggering during query + * execution. For other guardrails, say one triggering during compaction, failing does not make sense). + * + *

    Additionally, each individual setting should have a specific value (typically -1 for numeric settings), + * that allows to disable the corresponding guardrail. + * + *

    The default values for each guardrail settings should reflect what is mandated for DCaaS. + * + *

    For consistency, guardrails based on a simple numeric threshold should use the naming scheme + * {@code _warn_threshold} for soft limits and {@code _failure_threshold} for hard + * ones, and if the value has a unit, that unit should be added at the end (for instance, + * {@code _failure_threshold_in_kb}). For "boolean" guardrails that disable a feature, use + * {@code }. + */ +public class GuardrailsConfig +{ + public static final String INDEX_GUARDRAILS_TABLE_FAILURE_THRESHOLD = Config.PROPERTY_PREFIX + "index.guardrails.table_failure_threshold"; + public static final String INDEX_GUARDRAILS_TOTAL_FAILURE_THRESHOLD = Config.PROPERTY_PREFIX + "index.guardrails.total_failure_threshold"; + + public static final int NO_LIMIT = -1; + public static final int UNSET = -2; + public static final int DEFAULT_INDEXES_PER_TABLE_THRESHOLD = 10; + public static final int DEFAULT_INDEXES_TOTAL_THRESHOLD = 100; + + public volatile Long column_value_size_failure_threshold_in_kb; + public volatile Long columns_per_table_failure_threshold; + public volatile Long fields_per_udt_failure_threshold; + public volatile Long collection_size_warn_threshold_in_kb; + public volatile Long items_per_collection_warn_threshold; + public volatile Boolean read_before_write_list_operations_enabled; + + // Legacy 2i guardrail + public volatile Integer secondary_index_per_table_failure_threshold; + public volatile Integer sasi_indexes_per_table_failure_threshold; + // SAI indexes guardrail + public volatile Integer sai_indexes_per_table_failure_threshold; + public volatile Integer sai_indexes_total_failure_threshold; + public volatile Integer materialized_view_per_table_failure_threshold; + + public volatile Long tables_warn_threshold; + public volatile Long tables_failure_threshold; + // N.B. Not safe for concurrent modification + public volatile Set table_properties_disallowed; + public volatile Set table_properties_ignored; + + public volatile Boolean user_timestamps_enabled; + + public volatile Boolean logged_batch_enabled; + + public volatile Boolean truncate_table_enabled; + + public volatile Boolean counter_enabled; + + public volatile Set write_consistency_levels_disallowed; + + // For paging by bytes having a page bigger than this threshold will result in a failure + // For paging by rows the result will be silently cut short if it is bigger than the threshold + public volatile Integer page_size_failure_threshold_in_kb; + + // Limit number of terms and their cartesian product in IN query + public volatile Integer in_select_cartesian_product_failure_threshold; + public volatile Integer partition_keys_in_select_failure_threshold; + + // represent percentage of disk space, -1 means disabled + public volatile Integer disk_usage_percentage_warn_threshold; + public volatile Integer disk_usage_percentage_failure_threshold; + public volatile Long disk_usage_max_disk_size_in_gb; + + // When executing a scan, within or across a partition, we need to keep the + // tombstones seen in memory so we can return them to the coordinator, which + // will use them to make sure other replicas also know about the deleted rows. + // With workloads that generate a lot of tombstones, this can cause performance + // problems and even exaust the server heap. + // (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets) + // Adjust the thresholds here if you understand the dangers and want to + // scan more tombstones anyway. These thresholds may also be adjusted at runtime + // using the StorageService mbean. + public volatile Integer tombstone_warn_threshold; + public volatile Integer tombstone_failure_threshold; + + // Log WARN on any multiple-partition batch size that exceeds this value. 5kb per batch by default. + // Use caution when increasing the size of this threshold as it can lead to node instability. + public volatile Integer batch_size_warn_threshold_in_kb; + // Fail any multiple-partition batch that exceeds this value. The calculated default is 50kb (10x warn threshold). + public volatile Integer batch_size_fail_threshold_in_kb; + // Log WARN on any batches not of type LOGGED than span across more partitions than this limit. + public volatile Integer unlogged_batch_across_partitions_warn_threshold; + + public volatile Integer partition_size_warn_threshold_in_mb; + + /** + * If {@link DatabaseDescriptor#isEmulateDbaasDefaults()} is true, apply cloud defaults to guardrails settings that + * are not specified in yaml; otherwise, apply on-prem defaults to guardrails settings that are not specified in yaml; + */ + @VisibleForTesting + public void applyConfig() + { + // for read requests + enforceDefault(page_size_failure_threshold_in_kb, v -> page_size_failure_threshold_in_kb = v, NO_LIMIT, 512); + + enforceDefault(in_select_cartesian_product_failure_threshold, v -> in_select_cartesian_product_failure_threshold = v, NO_LIMIT, 25); + enforceDefault(partition_keys_in_select_failure_threshold, v -> partition_keys_in_select_failure_threshold = v, NO_LIMIT, 20); + + enforceDefault(tombstone_warn_threshold, v -> tombstone_warn_threshold = v, 1000, 1000); + enforceDefault(tombstone_failure_threshold, v -> tombstone_failure_threshold = v, 100000, 100000); + + // for write requests + enforceDefault(logged_batch_enabled, v -> logged_batch_enabled = v, true, true); + enforceDefault(batch_size_warn_threshold_in_kb, v -> batch_size_warn_threshold_in_kb = v, 64, 64); + enforceDefault(batch_size_fail_threshold_in_kb, v -> batch_size_fail_threshold_in_kb = v, 640, 640); + enforceDefault(unlogged_batch_across_partitions_warn_threshold, v -> unlogged_batch_across_partitions_warn_threshold = v, 10, 10); + + enforceDefault(truncate_table_enabled, v -> truncate_table_enabled = v, true, true); + + enforceDefault(user_timestamps_enabled, v -> user_timestamps_enabled = v, true, true); + + enforceDefault(column_value_size_failure_threshold_in_kb, v -> column_value_size_failure_threshold_in_kb = v, -1L, 5 * 1024L); + + enforceDefault(read_before_write_list_operations_enabled, v -> read_before_write_list_operations_enabled = v, true, false); + + // We use a LinkedHashSet just for the sake of preserving the ordering in error messages + enforceDefault(write_consistency_levels_disallowed, + v -> write_consistency_levels_disallowed = ImmutableSet.copyOf(v), + Collections.emptySet(), + new LinkedHashSet<>(Arrays.asList("ANY", "ONE", "LOCAL_ONE"))); + + // for schema + enforceDefault(counter_enabled, v -> counter_enabled = v, true, true); + + enforceDefault(fields_per_udt_failure_threshold, v -> fields_per_udt_failure_threshold = v, -1L, 10L); + enforceDefault(collection_size_warn_threshold_in_kb, v -> collection_size_warn_threshold_in_kb = v, -1L, 5 * 1024L); + enforceDefault(items_per_collection_warn_threshold, v -> items_per_collection_warn_threshold = v, -1L, 20L); + + enforceDefault(columns_per_table_failure_threshold, v -> columns_per_table_failure_threshold = v, -1L, 50L); + enforceDefault(secondary_index_per_table_failure_threshold, v -> secondary_index_per_table_failure_threshold = v, NO_LIMIT, 1); + enforceDefault(sasi_indexes_per_table_failure_threshold, v -> sasi_indexes_per_table_failure_threshold = v, NO_LIMIT, 0); + enforceDefault(materialized_view_per_table_failure_threshold, v -> materialized_view_per_table_failure_threshold = v, NO_LIMIT, 2); + enforceDefault(tables_warn_threshold, v -> tables_warn_threshold = v, -1L, 100L); + enforceDefault(tables_failure_threshold, v -> tables_failure_threshold = v, -1L, 200L); + + enforceDefault(table_properties_disallowed, + v -> table_properties_disallowed = ImmutableSet.copyOf(v), + Collections.emptySet(), + Collections.emptySet()); + + enforceDefault(table_properties_ignored, + v -> table_properties_ignored = ImmutableSet.copyOf(v), + Collections.emptySet(), + new LinkedHashSet<>(TableAttributes.allKeywords().stream() + .sorted() + .filter(p -> !p.equals("default_time_to_live")) + .collect(Collectors.toList()))); + + // for node status + enforceDefault(disk_usage_percentage_warn_threshold, v -> disk_usage_percentage_warn_threshold = v, NO_LIMIT, 70); + enforceDefault(disk_usage_percentage_failure_threshold, v -> disk_usage_percentage_failure_threshold = v, NO_LIMIT, 80); + enforceDefault(disk_usage_max_disk_size_in_gb, v -> disk_usage_max_disk_size_in_gb = v, (long) NO_LIMIT, (long) NO_LIMIT); + + enforceDefault(partition_size_warn_threshold_in_mb, v -> partition_size_warn_threshold_in_mb = v, 100, 100); + + // SAI Table Failure threshold (maye be overridden via system property) + Integer overrideTableFailureThreshold = Integer.getInteger(INDEX_GUARDRAILS_TABLE_FAILURE_THRESHOLD, UNSET); + if (overrideTableFailureThreshold != UNSET) + sai_indexes_per_table_failure_threshold = overrideTableFailureThreshold; + enforceDefault(sai_indexes_per_table_failure_threshold, v -> sai_indexes_per_table_failure_threshold = v, DEFAULT_INDEXES_PER_TABLE_THRESHOLD, DEFAULT_INDEXES_PER_TABLE_THRESHOLD); + + // SAI Table Failure threshold (maye be overridden via system property) + Integer overrideTotalFailureThreshold = Integer.getInteger(INDEX_GUARDRAILS_TOTAL_FAILURE_THRESHOLD, UNSET); + if (overrideTotalFailureThreshold != UNSET) + sai_indexes_total_failure_threshold = overrideTotalFailureThreshold; + enforceDefault(sai_indexes_total_failure_threshold, v -> sai_indexes_total_failure_threshold = v, DEFAULT_INDEXES_TOTAL_THRESHOLD, DEFAULT_INDEXES_TOTAL_THRESHOLD); + } + + /** + * Validate that the value provided for each guardrail setting is valid. + * + * @throws ConfigurationException if any of the settings has an invalid setting. + */ + public void validate() + { + validateStrictlyPositiveInteger(column_value_size_failure_threshold_in_kb, + "column_value_size_failure_threshold_in_kb"); + + validateStrictlyPositiveInteger(columns_per_table_failure_threshold, + "columns_per_table_failure_threshold"); + + validateStrictlyPositiveInteger(fields_per_udt_failure_threshold, + "fields_per_udt_failure_threshold"); + + validateStrictlyPositiveInteger(collection_size_warn_threshold_in_kb, + "collection_size_warn_threshold_in_kb"); + + validateStrictlyPositiveInteger(items_per_collection_warn_threshold, + "items_per_collection_warn_threshold"); + + validateStrictlyPositiveInteger(tables_warn_threshold, "tables_warn_threshold"); + validateStrictlyPositiveInteger(tables_failure_threshold, "tables_failure_threshold"); + validateWarnLowerThanFail(tables_warn_threshold, tables_failure_threshold, "tables"); + + validateDisallowedTableProperties(); + validateIgnoredTableProperties(); + + validateStrictlyPositiveInteger(page_size_failure_threshold_in_kb, "page_size_failure_threshold_in_kb"); + + validateStrictlyPositiveInteger(partition_size_warn_threshold_in_mb, "partition_size_warn_threshold_in_mb"); + + validateStrictlyPositiveInteger(partition_keys_in_select_failure_threshold, "partition_keys_in_select_failure_threshold"); + + validateStrictlyPositiveInteger(in_select_cartesian_product_failure_threshold, "in_select_cartesian_product_failure_threshold"); + + validateDiskUsageThreshold(); + + validateTombstoneThreshold(tombstone_warn_threshold, tombstone_failure_threshold); + + validateBatchSizeThreshold(batch_size_warn_threshold_in_kb, batch_size_fail_threshold_in_kb); + validateStrictlyPositiveInteger(unlogged_batch_across_partitions_warn_threshold, "unlogged_batch_across_partitions_warn_threshold"); + + for (String rawCL : write_consistency_levels_disallowed) + { + try + { + ConsistencyLevel.fromString(rawCL); + } + catch (Exception e) + { + throw new ConfigurationException(format("Invalid value for write_consistency_levels_disallowed guardrail: " + + "'%s' does not parse as a Consistency Level", rawCL)); + } + } + } + + /** + * This validation method should only be called after {@link DatabaseDescriptor#createAllDirectories()} has been called. + */ + public void validateAfterDataDirectoriesExist() + { + validateDiskUsageMaxSize(); + } + + @VisibleForTesting + public void validateDiskUsageMaxSize() + { + long totalDiskSizeInGb = 0L; + for (Directories.DataDirectory directory : Directories.dataDirectories.getAllDirectories()) + { + totalDiskSizeInGb += SizeUnit.BYTES.toGigaBytes(directory.getTotalSpace()); + } + + if (totalDiskSizeInGb == 0L) + { + totalDiskSizeInGb = Long.MAX_VALUE; + } + validatePositiveNumeric(disk_usage_max_disk_size_in_gb, totalDiskSizeInGb, false, "disk_usage_max_disk_size_in_gb"); + } + + /** + * Enforce default value based on {@link DatabaseDescriptor#isEmulateDbaasDefaults()} if + * it's not specified in yaml + * + * @param current current config value defined in yaml + * @param optionSetter setter to updated given config + * @param onPremDefault default value for on-prem + * @param dbaasDefault default value for constellation DB-as-a-service + * @param + */ + private static void enforceDefault(T current, Consumer optionSetter, T onPremDefault, T dbaasDefault) + { + if (current != null) + return; + + optionSetter.accept(DatabaseDescriptor.isEmulateDbaasDefaults() ? dbaasDefault : onPremDefault); + } + + /** + * @return true if given disk usage threshold disables disk usage guardrail + */ + public static boolean diskUsageGuardrailDisabled(double value) + { + return value < 0; + } + + /** + * Validate that the values provided for disk usage are valid. + * + * @throws ConfigurationException if any of the settings has an invalid setting. + */ + @VisibleForTesting + public void validateDiskUsageThreshold() + { + validatePositiveNumeric(disk_usage_percentage_warn_threshold, 100, false, "disk_usage_percentage_warn_threshold"); + validatePositiveNumeric(disk_usage_percentage_failure_threshold, 100, false, "disk_usage_percentage_failure_threshold"); + validateWarnLowerThanFail(disk_usage_percentage_warn_threshold, disk_usage_percentage_failure_threshold, "disk_usage_percentage"); + } + + public void validateTombstoneThreshold(long warnThreshold, long failureThreshold) + { + validateStrictlyPositiveInteger(warnThreshold, "tombstone_warn_threshold"); + validateStrictlyPositiveInteger(failureThreshold, "tombstone_failure_threshold"); + validateWarnLowerThanFail(warnThreshold, failureThreshold, "tombstone_threshold"); + } + + private void validateDisallowedTableProperties() + { + Set diff = Sets.difference(table_properties_disallowed.stream().map(String::toLowerCase).collect(Collectors.toSet()), + TableAttributes.allKeywords()); + + if (!diff.isEmpty()) + throw new ConfigurationException(format("Invalid value for table_properties_disallowed guardrail: " + + "'%s' do not parse as valid table properties", diff.toString())); + } + + private void validateIgnoredTableProperties() + { + Set diff = Sets.difference(table_properties_ignored.stream().map(String::toLowerCase).collect(Collectors.toSet()), + TableAttributes.allKeywords()); + + if (!diff.isEmpty()) + throw new ConfigurationException(format("Invalid value for table_properties_ignored guardrail: " + + "'%s' do not parse as valid table properties", diff.toString())); + } + + private void validateStrictlyPositiveInteger(long value, String name) + { + // We use 'long' for generality, but most numeric guardrail cannot effectively be more than a 'int' for various + // internal reasons. Not that any should ever come close in practice ... + // Also, in most cases, zero does not make sense (allowing 0 tables or columns is not exactly useful). + validatePositiveNumeric(value, Integer.MAX_VALUE, false, name); + } + + private void validatePositiveNumeric(long value, long maxValue, boolean allowZero, String name) + { + if (value > maxValue) + throw new ConfigurationException(format("Invalid value %d for guardrail %s: maximum allowed value is %d", + value, name, maxValue)); + + if (value == 0 && !allowZero) + throw new ConfigurationException(format("Invalid value for guardrail %s: 0 is not allowed", name)); + + // We allow -1 as a general "disabling" flag. But reject anything lower to avoid mistakes. + if (value < -1L) + throw new ConfigurationException(format("Invalid value %d for guardrail %s: negative values are not " + + "allowed, outside of -1 which disables the guardrail", + value, name)); + } + + private void validateWarnLowerThanFail(long warnValue, long failValue, String guardName) + { + if (warnValue == -1 || failValue == -1) + return; + + if (failValue < warnValue) + throw new ConfigurationException(format("The warn threshold %d for the %s guardrail should be lower " + + "than the failure threshold %d", + warnValue, guardName, failValue)); + } + + public void setTombstoneFailureThreshold(int threshold) + { + validateTombstoneThreshold(tombstone_warn_threshold, threshold); + tombstone_failure_threshold = threshold; + } + + public void setTombstoneWarnThreshold(int threshold) + { + validateTombstoneThreshold(threshold, tombstone_failure_threshold); + tombstone_warn_threshold = threshold; + } + + + public void validateBatchSizeThreshold(long warnThreshold, long failureThreshold) + { + validateStrictlyPositiveInteger(warnThreshold, "batch_size_warn_threshold_in_kb"); + validateStrictlyPositiveInteger(failureThreshold, "batch_size_fail_threshold_in_kb"); + validateWarnLowerThanFail(warnThreshold, failureThreshold, "batch_size_threshold"); + } + + public int getBatchSizeWarnThreshold() + { + return batch_size_warn_threshold_in_kb * 1024; + } + + public int getBatchSizeFailThreshold() + { + return batch_size_fail_threshold_in_kb * 1024; + } + + public void setBatchSizeWarnThresholdInKB(int threshold) + { + validateBatchSizeThreshold(threshold, batch_size_fail_threshold_in_kb); + batch_size_warn_threshold_in_kb = threshold; + } + + public void setBatchSizeFailThresholdInKB(int threshold) + { + validateBatchSizeThreshold(batch_size_warn_threshold_in_kb, threshold); + batch_size_fail_threshold_in_kb = threshold; + } +} diff --git a/src/java/org/apache/cassandra/hints/HintsDescriptor.java b/src/java/org/apache/cassandra/hints/HintsDescriptor.java index 1979637779e4..d9c11ab770f0 100644 --- a/src/java/org/apache/cassandra/hints/HintsDescriptor.java +++ b/src/java/org/apache/cassandra/hints/HintsDescriptor.java @@ -64,7 +64,8 @@ final class HintsDescriptor static final int VERSION_30 = 1; static final int VERSION_40 = 2; - static final int CURRENT_VERSION = VERSION_40; + static final int VERSION_SG_10 = 100; + static final int CURRENT_VERSION = VERSION_SG_10; static final String COMPRESSION = "compression"; static final String ENCRYPTION = "encryption"; @@ -220,6 +221,8 @@ static int messagingVersion(int hintsVersion) return MessagingService.VERSION_30; case VERSION_40: return MessagingService.VERSION_40; + case VERSION_SG_10: + return MessagingService.VERSION_SG_10; default: throw new AssertionError(); } diff --git a/src/java/org/apache/cassandra/index/Index.java b/src/java/org/apache/cassandra/index/Index.java index e9d3d3c3d332..4ecdb09c089a 100644 --- a/src/java/org/apache/cassandra/index/Index.java +++ b/src/java/org/apache/cassandra/index/Index.java @@ -20,18 +20,24 @@ */ package org.apache.cassandra.index; +import java.nio.ByteBuffer; import java.util.Collection; +import java.util.Collections; import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; -import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; -import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.*; -import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.partitions.PartitionIterator; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; @@ -39,11 +45,16 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.index.internal.CollatedViewIndexBuilder; import org.apache.cassandra.index.transactions.IndexTransaction; +import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.ReducingKeyIterator; +import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; +import org.apache.cassandra.schema.TableMetadata; + /** * Consisting of a top level Index interface and two sub-interfaces which handle read and write operations, @@ -163,7 +174,7 @@ public boolean supportsReads() */ interface IndexBuildingSupport { - SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs, Set indexes, Collection sstables); + SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs, Set indexes, Collection sstables, boolean isFullRebuild); } /** @@ -173,7 +184,7 @@ interface IndexBuildingSupport public static class CollatedViewIndexBuildingSupport implements IndexBuildingSupport { @SuppressWarnings("resource") - public SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs, Set indexes, Collection sstables) + public SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs, Set indexes, Collection sstables, boolean isFullRebuild) { return new CollatedViewIndexBuilder(cfs, indexes, new ReducingKeyIterator(sstables), sstables); } @@ -315,11 +326,11 @@ default public Callable getPreJoinTask(boolean hadBootstrap) * Get flush observer to observe partition/cell events generated by flushing SSTable (memtable flush or compaction). * * @param descriptor The descriptor of the sstable observer is requested for. - * @param opType The type of the operation which requests observer e.g. memtable flush or compaction. + * @param tracker The {@link LifecycleNewTracker} associated with the SSTable being written * * @return SSTable flush observer. */ - default SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationType opType) + default SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker) { return null; } @@ -364,6 +375,23 @@ default SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationTy */ public AbstractType customExpressionValueType(); + /** + * If the index supports custom search expressions using the + * {@code SELECT * FROM table WHERE expr(index_name, expression)} syntax, this method should return a new + * {@link RowFilter.CustomExpression} for the specified expression value. Index implementations may provide their + * own implementations using method {@link RowFilter.CustomExpression#isSatisfiedBy(TableMetadata, DecoratedKey, Row)} + * to filter reconciled rows in the coordinator. Otherwise, the default implementation will accept all rows. + * See DB-2185 and DSP-16537 for further details. + * + * @param metadata the indexed table metadata + * @param value the custom expression value + * @return a custom index expression for the specified value + */ + default RowFilter.CustomExpression customExpressionFor(TableMetadata metadata, ByteBuffer value) + { + return new RowFilter.CustomExpression(metadata, getIndexMetadata(), value); + } + /** * Transform an initial RowFilter into the filter that will still need to applied * to a set of Rows after the index has performed it's initial scan. @@ -386,6 +414,17 @@ default SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationTy */ public long getEstimatedResultRows(); + /** + * Check if current index is queryable based on the index status. + * + * @param status current status of the index + * @return true if index should be queryable, false if index should be non-queryable + */ + default boolean isQueryable(Status status) + { + return true; + } + /* * Input validation */ @@ -401,6 +440,16 @@ default SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationTy */ public void validate(PartitionUpdate update) throws InvalidRequestException; + /** + * Returns the SSTable-attached {@link Component}s created by this index. + * + * @return the SSTable components created by this index + */ + default Set getComponents() + { + return Collections.emptySet(); + } + /* * Update processing */ @@ -416,6 +465,7 @@ default SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationTy * @param ctx WriteContext spanning the update operation * @param transactionType indicates what kind of update is being performed on the base data * i.e. a write time insert/update/delete or the result of compaction + * @param memtable The current memtable that is the source of the updates * @return the newly created indexer or {@code null} if the index is not interested by the update * (this could be because the index doesn't care about that particular partition, doesn't care about * that type of transaction, ...). @@ -424,7 +474,8 @@ public Indexer indexerFor(DecoratedKey key, RegularAndStaticColumns columns, int nowInSec, WriteContext ctx, - IndexTransaction.Type transactionType); + IndexTransaction.Type transactionType, + Memtable memtable); /** * Listener for processing events emitted during a single partition update. @@ -562,22 +613,6 @@ default boolean supportsReplicaFilteringProtection(RowFilter rowFilter) return true; } - /** - * Return a function which performs post processing on the results of a partition range read command. - * In future, this may be used as a generalized mechanism for transforming results on the coordinator prior - * to returning them to the caller. - * - * This is used on the coordinator during execution of a range command to perform post - * processing of merged results obtained from the necessary replicas. This is the only way in which results are - * transformed in this way but this may change over time as usage is generalized. - * See CASSANDRA-8717 for further discussion. - * - * The function takes a PartitionIterator of the results from the replicas which has already been collated - * and reconciled, along with the command being executed. It returns another PartitionIterator containing the results - * of the transformation (which may be the same as the input if the transformation is a no-op). - */ - public BiFunction postProcessorFor(ReadCommand command); - /** * Factory method for query time search helper. * @@ -593,10 +628,302 @@ default boolean supportsReplicaFilteringProtection(RowFilter rowFilter) */ public interface Searcher { + /** + * Returns the {@link ReadCommand} for which this searcher has been created. + * + * @return the base read command + */ + ReadCommand command(); + /** * @param executionController the collection of OpOrder.Groups which the ReadCommand is being performed under. * @return partitions from the base table matching the criteria of the search. */ public UnfilteredPartitionIterator search(ReadExecutionController executionController); + + /** + * Replica filtering protection may fetch data that doesn't match query conditions. + * + * On coordinator, we need to filter the replicas' responses again. + * + * @return filtered response that satisfied query conditions + */ + default PartitionIterator filterReplicaFilteringProtection(PartitionIterator fullResponse) + { + return command().rowFilter().filter(fullResponse, command().metadata(), command().nowInSec()); + } + } + + /** + * Class providing grouped operations for indexes that communicate with each other. + * + * Index implementations should provide a {@code Group} implementation calling to + * {@link SecondaryIndexManager#registerIndex(Index, Object, Supplier)} during index registering + * at {@link #register(IndexRegistry)} method. + */ + interface Group + { + /** + * Returns the indexes that are members of this group. + * + * @return the indexes that are members of this group + */ + Set getIndexes(); + + /** + * Adds the specified {@link Index} as a member of this group. + * + * @param index the index to be added + */ + void addIndex(Index index); + + /** + * Removes the specified {@link Index} from the members of this group. + * + * @param index the index to be removed + */ + void removeIndex(Index index); + + /** + * Returns if this group contains the specified {@link Index}. + * + * @param index the index to be removed + * @return {@code true} if this group contains {@code index}, {@code false} otherwise + */ + boolean containsIndex(Index index); + + /** + * Creates an new {@code Indexer} object for updates to a given partition. + * + * @param indexSelector a predicate selecting the targeted members + * @param key key of the partition being modified + * @param columns the regular and static columns the created indexer will have to deal with. + * This can be empty as an update might only contain partition, range and row deletions, but + * the indexer is guaranteed to not get any cells for a column that is not part of {@code columns}. + * @param nowInSec current time of the update operation + * @param ctx WriteContext spanning the update operation + * @param transactionType indicates what kind of update is being performed on the base data + * i.e. a write time insert/update/delete or the result of compaction + * @param memtable the {@link Memtable} to which the updates are being applied or {@code null} + * if the source of the updates is an existing {@link SSTable} + * + * @return the newly created indexer or {@code null} if the index is not interested by the update + * (this could be because the index doesn't care about that particular partition, doesn't care about + * that type of transaction, ...). + */ + Indexer indexerFor(Predicate indexSelector, + DecoratedKey key, + RegularAndStaticColumns columns, + int nowInSec, + WriteContext ctx, + IndexTransaction.Type transactionType, + Memtable memtable); + + /** + * Returns a new {@link QueryPlan} for the specified {@link RowFilter}, or {@code null} if none of the indexes in + * this group supports the expression in the row filter. + * + * @param rowFilter a row filter + * @return a new query plan for the specified {@link RowFilter} if it's supported, {@code null} otherwise + */ + @Nullable + QueryPlan queryPlanFor(RowFilter rowFilter); + + /** + * Get flush observer to observe partition/cell events generated by flushing SSTable (memtable flush or compaction). + * + * @param descriptor The descriptor of the sstable observer is requested for. + * @param tracker The {@link LifecycleNewTracker} associated with the SSTable being written + * @param tableMetadata The immutable metadata of the table at the moment the SSTable is flushed + * + * @return SSTable flush observer. + */ + SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker, TableMetadata tableMetadata); + + /** + * @param type index transaction type + * @return true if index will be able to handle given index transaction type + */ + default boolean handles(IndexTransaction.Type type) + { + return true; + } + + /** + * Called when the table associated with this group has been invalidated. Implementations + * should dispose of any resources tied to the lifecycle of the {@link Group}. + */ + default void invalidate() { } + + /** + * Returns the SSTable-attached {@link Component}s created by this index group. + * + * @return the SSTable components created by this group + */ + Set getComponents(); + + /** + * @return true if this index group is capable of supporting multiple contains restrictions, false otherwise + */ + default boolean supportsMultipleContains() + { + return false; + } + + /** + * @return true is this index group supports disjunction queries of "a = 1 OR a = 2" or "a IN (1, 2)" + */ + default boolean supportsDisjunction() + { + return false; + } + } + + /** + * Specifies a set of compatible indexes to be used with a query according to its {@link RowFilter}, ignoring data + * ranges, limits, etc. All the indexes in it should belong to the same {@link Group}. + *

    + * It's created by {@link Group#queryPlanFor} from the {@link RowFilter} that is common to all the subcommands of a + * user query's {@link ReadCommand}, so it can be reused by those subcommands along the cluster nodes. The + * {@link #searcherFor(ReadCommand)} method provides the {@link Searcher} object to read the index for each + * particular (sub)command. + */ + interface QueryPlan extends Comparable + { + /** + * Returns the indexes selected by this query plan, all of them belonging to the same {@link Group}. + * + * It should never be empty. + * + * @return the indexes selected by this query plan, which is never empty + */ + Set getIndexes(); + + /** + * Returns the first index in this plan. + * + * @return the first index + */ + @Nonnull + default Index getFirst() + { + return getIndexes().iterator().next(); + } + + /** + * Return an estimate of the number of results this plan is expected to return for any given {@link ReadCommand} + * that it can be used to answer. Used by {@link SecondaryIndexManager#getBestIndexQueryPlanFor(RowFilter)} + * to determine the {@link Group} with the most selective plan for a given {@link RowFilter}. + * Additionally, this is also used by StorageProxy.estimateResultsPerRange to calculate the initial concurrency + * factor for range requests + * + * @return the estimated average number of results a Searcher may return for any given command + */ + default long getEstimatedResultRows() + { + // CQL only supports AND expressions, so the estimated number of results for multiple indexes will be the + // the lowest of the estimates for each index + return getIndexes().stream() + .mapToLong(Index::getEstimatedResultRows) + .min() + .orElseThrow(AssertionError::new); // registered groups are never empty + } + + /** + * Used to determine whether to estimate initial concurrency during remote range reads. Default is true, each + * implementation must override this method if they choose a different strategy (e.g. StorageAttachedIndexQueryPlan). + * + * @return true if the {@link QueryPlan} should estimate initial concurrency, false otherwise + */ + default boolean shouldEstimateInitialConcurrency() + { + return true; + } + + @Override + default int compareTo(QueryPlan other) + { + // initially, we prefer the plan with less estimated results + int results = Long.compare(getEstimatedResultRows(), other.getEstimatedResultRows()); + if (results != 0) + return results; + + // In case of having the same number of estimated results, we favour the plan that involves more indexes. + // This way, we honour the possible absence of ALLOW FILTERING in the CQL query. Also, this criteria should + // not break the transitivity of this method because the estimated number of results for a plan is the + // minimum of the estimates of its members. + return Integer.compare(getIndexes().size(), other.getIndexes().size()); + } + + /** + * Used to validate the various parameters of a supplied {@link ReadCommand} against the indexes in this plan. + * + * @param command a ReadCommand whose parameters are to be verified + * @throws InvalidRequestException if the details of the command fail to meet the validation rules of the + * indexes in the query plan + */ + default void validate(ReadCommand command) throws InvalidRequestException + { + getIndexes().forEach(i -> i.validate(command)); + } + + /** + * Factory method for query time search helper. + * + * @param command the read command being executed + * @return an Searcher with which to perform the supplied command + */ + Searcher searcherFor(ReadCommand command); + + /** + * Return a function which performs post processing on the results of a partition range read command. + * In future, this may be used as a generalized mechanism for transforming results on the coordinator prior + * to returning them to the caller. + * + * This is used on the coordinator during execution of a range command to perform post + * processing of merged results obtained from the necessary replicas. This is the only way in which results are + * transformed in this way but this may change over time as usage is generalized. + * See CASSANDRA-8717 for further discussion. + * + * The function takes a PartitionIterator of the results from the replicas which has already been collated + * and reconciled, along with the command being executed. It returns another PartitionIterator containing the results + * of the transformation (which may be the same as the input if the transformation is a no-op). + */ + default Function postProcessor() + { + return partitions -> partitions; + } + + /** + * Transform an initial {@link RowFilter} into the filter that will still need to applied to a set of Rows after + * the index has performed it's initial scan. + * + * Used in {@link ReadCommand#executeLocally(ReadExecutionController)} to reduce the amount of filtering performed on the + * results of the index query. + * + * @return the (hopefully) reduced filter that would still need to be applied after + * the index was used to narrow the initial result set + */ + RowFilter postIndexQueryFilter(); + + /** + * @return true if the indexes in this plan support querying multiple vnode ranges at once. + */ + default boolean supportsMultiRangeReadCommand() + { + return false; + } + } + + /* + * Status of index used to determine queryability + */ + enum Status + { + UNKNOWN, + FULL_REBUILD_STARTED, + BUILD_FAILED, + BUILD_SUCCEEDED, + DROPPED } } diff --git a/src/java/org/apache/cassandra/index/IndexRegistry.java b/src/java/org/apache/cassandra/index/IndexRegistry.java index 0cf1cbb6c2a5..c193c4cef01f 100644 --- a/src/java/org/apache/cassandra/index/IndexRegistry.java +++ b/src/java/org/apache/cassandra/index/IndexRegistry.java @@ -23,19 +23,30 @@ import java.util.Collection; import java.util.Collections; import java.util.Optional; +import java.util.Set; import java.util.concurrent.Callable; import java.util.function.BiFunction; +import java.util.function.Predicate; +import java.util.function.Supplier; + +import javax.annotation.Nullable; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.*; +import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.partitions.PartitionIterator; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.index.transactions.IndexTransaction; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.TableMetadata; @@ -56,19 +67,20 @@ public interface IndexRegistry public static final IndexRegistry EMPTY = new IndexRegistry() { @Override - public void unregisterIndex(Index index) + public void registerIndex(Index index, Object groupKey, Supplier groupSupplier) { } @Override - public void registerIndex(Index index) + public Collection listIndexes() { + return Collections.emptyList(); } @Override - public Collection listIndexes() + public Collection listIndexGroups() { - return Collections.emptyList(); + return Collections.emptySet(); } @Override @@ -173,23 +185,70 @@ public void validate(PartitionUpdate update) throws InvalidRequestException { } - public Indexer indexerFor(DecoratedKey key, RegularAndStaticColumns columns, int nowInSec, WriteContext ctx, IndexTransaction.Type transactionType) + public Indexer indexerFor(DecoratedKey key, RegularAndStaticColumns columns, int nowInSec, WriteContext ctx, IndexTransaction.Type transactionType, Memtable memtable) { return null; } - public BiFunction postProcessorFor(ReadCommand command) + public Searcher searcherFor(ReadCommand command) { return null; } + }; - public Searcher searcherFor(ReadCommand command) + Index.Group group = new Index.Group() + { + @Override + public Set getIndexes() + { + return Collections.singleton(index); + } + + @Override + public void addIndex(Index index) + { + } + + @Override + public void removeIndex(Index index) + { + } + + @Override + public boolean containsIndex(Index i) + { + return index == i; + } + + @Nullable + @Override + public Index.Indexer indexerFor(Predicate indexSelector, DecoratedKey key, RegularAndStaticColumns columns, int nowInSec, WriteContext ctx, IndexTransaction.Type transactionType, Memtable memtable) + { + return null; + } + + @Nullable + @Override + public Index.QueryPlan queryPlanFor(RowFilter rowFilter) + { + return null; + } + + @Nullable + @Override + public SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker, TableMetadata tableMetadata) + { + return null; + } + + @Override + public Set getComponents() { return null; } }; - public void registerIndex(Index index) + public void registerIndex(Index index, Object groupKey, Supplier groupSupplier) { } @@ -207,6 +266,12 @@ public Collection listIndexes() return Collections.singletonList(index); } + @Override + public Collection listIndexGroups() + { + return Collections.singletonList(group); + } + public Optional getBestIndexFor(RowFilter.Expression expression) { return Optional.empty(); @@ -218,8 +283,12 @@ public void validate(PartitionUpdate update) } }; - void registerIndex(Index index); - void unregisterIndex(Index index); + default void registerIndex(Index index) + { + registerIndex(index, index, () -> new SingletonIndexGroup(index)); + } + public void registerIndex(Index index, Object groupKey, Supplier groupSupplier); + Collection listIndexGroups(); Index getIndex(IndexMetadata indexMetadata); Collection listIndexes(); diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexBuilder.java b/src/java/org/apache/cassandra/index/SecondaryIndexBuilder.java index 73dc3345a250..eeaf0ce81d70 100644 --- a/src/java/org/apache/cassandra/index/SecondaryIndexBuilder.java +++ b/src/java/org/apache/cassandra/index/SecondaryIndexBuilder.java @@ -17,12 +17,12 @@ */ package org.apache.cassandra.index; -import org.apache.cassandra.db.compaction.CompactionInfo; +import org.apache.cassandra.db.compaction.AbstractTableOperation; /** * Manages building an entire index from column family data. Runs on to compaction manager. */ -public abstract class SecondaryIndexBuilder extends CompactionInfo.Holder +public abstract class SecondaryIndexBuilder extends AbstractTableOperation { public abstract void build(); diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java index e9b22ef6d561..7f6a7f4f5056 100644 --- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java +++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java @@ -22,9 +22,13 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Strings; @@ -33,7 +37,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.primitives.Longs; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -50,32 +53,42 @@ import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.concurrent.Stage; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.PageSize; import org.apache.cassandra.cql3.statements.schema.IndexTarget; import org.apache.cassandra.db.*; import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.db.lifecycle.View; -import org.apache.cassandra.db.marshal.ValueAccessor; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.index.Index.IndexBuildingSupport; +import org.apache.cassandra.exceptions.ReadFailureException; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.index.internal.CassandraIndex; import org.apache.cassandra.index.transactions.*; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.locator.Endpoints; +import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.notifications.INotification; import org.apache.cassandra.notifications.INotificationConsumer; import org.apache.cassandra.notifications.SSTableAddedNotification; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.Indexes; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.pager.SinglePartitionPager; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.concurrent.Refs; +import org.json.simple.JSONValue; import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; import static org.apache.cassandra.utils.ExecutorUtils.shutdown; @@ -132,6 +145,11 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum // default page size (in rows) when rebuilding the index for a whole partition public static final int DEFAULT_PAGE_SIZE = 10000; + // store per-endpoint index status: the key of inner map is identifier "keyspace.index" + public static final Map> peerIndexStatus = new ConcurrentHashMap<>(); + // executes index status propagation task asynchronously to avoid potential deadlock on SIM + private static final ExecutorService statusPropagationExecutor = Executors.newSingleThreadExecutor(); + /** * All registered indexes. */ @@ -152,6 +170,11 @@ public class SecondaryIndexManager implements IndexRegistry, INotificationConsum */ private final Map writableIndexes = Maps.newConcurrentMap(); + /** + * The groups of all the registered indexes + */ + private final Map indexGroups = Maps.newConcurrentMap(); + /** * The count of pending index builds for each index. */ @@ -219,9 +242,15 @@ private synchronized Future createIndex(IndexMetadata indexDef, boolean isNew markIndexesBuilding(ImmutableSet.of(index), true, isNewCF); + return buildIndex(index); + } + + @VisibleForTesting + public Future buildIndex(final Index index) + { Callable initialBuildTask = null; // if the index didn't register itself, we can probably assume that no initialization needs to happen - if (indexes.containsKey(indexDef.name)) + if (indexes.containsKey(index.getIndexMetadata().name)) { try { @@ -267,7 +296,7 @@ public void onSuccess(Object o) * Adds and builds a index * * @param indexDef the IndexMetadata describing the index - * @param isNewCF true if the index is added as part of a new table/columnfamily (i.e. loading a CF at startup), + * @param isNewCF true if the index is added as part of a new table/columnfamily (i.e. loading a CF at startup), * false for all other cases (i.e. newly added index) */ public synchronized Future addIndex(IndexMetadata indexDef, boolean isNewCF) @@ -278,6 +307,22 @@ public synchronized Future addIndex(IndexMetadata indexDef, boolean isNewCF) return createIndex(indexDef, isNewCF); } + /** + * Throws an {@link IndexNotAvailableException} if any of the indexes in the specified {@link Index.QueryPlan} is + * not queryable, as it's defined by {@link #isIndexQueryable(Index)}. + * + * @param queryPlan a query plan + * @throws IndexNotAvailableException if the query plan has any index that is not queryable + */ + public void checkQueryability(Index.QueryPlan queryPlan) + { + for (Index index : queryPlan.getIndexes()) + { + if (!isIndexQueryable(index)) + throw new IndexNotAvailableException(index); + } + } + /** * Checks if the specified index is queryable. * @@ -384,7 +429,7 @@ public void rebuildIndexesBlocking(Set indexNames) // Once we are tracking new writes, flush any memtable contents to not miss them from the sstable-based rebuild if (needsFlush) - baseCfs.forceBlockingFlush(); + baseCfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.INDEX_BUILD_STARTED); // Now that we are tracking new writes and we haven't left untracked contents on the memtables, we are ready to // index the sstables @@ -497,7 +542,7 @@ private void buildIndexesBlocking(Collection sstables, Set { logger.info("Submitting index {} of {} for data in {}", isFullRebuild ? "recovery" : "build", - indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")), + commaSeparated(indexes), sstables.stream().map(SSTableReader::toString).collect(Collectors.joining(","))); // Group all building tasks @@ -515,7 +560,7 @@ private void buildIndexesBlocking(Collection sstables, Set List> futures = new ArrayList<>(byType.size()); byType.forEach((buildingSupport, groupedIndexes) -> { - SecondaryIndexBuilder builder = buildingSupport.getIndexBuildTask(baseCfs, groupedIndexes, sstables); + SecondaryIndexBuilder builder = buildingSupport.getIndexBuildTask(baseCfs, groupedIndexes, sstables, isFullRebuild); final SettableFuture build = SettableFuture.create(); Futures.addCallback(CompactionManager.instance.submitIndexBuild(builder), new FutureCallback() { @@ -624,10 +669,11 @@ private String getIndexNames(Set indexes) * * @param indexes the index to be marked as building * @param isFullRebuild {@code true} if this method is invoked as a full index rebuild, {@code false} otherwise - * @param isNewCF {@code true} if this method is invoked when initializing a new table/columnfamily (i.e. loading a CF at startup), + * @param isNewCF {@code true} if this method is invoked when initializing a new table/columnfamily (i.e. loading a CF at startup), * {@code false} for all other cases (i.e. newly added index) */ - private synchronized void markIndexesBuilding(Set indexes, boolean isFullRebuild, boolean isNewCF) + @VisibleForTesting + public synchronized void markIndexesBuilding(Set indexes, boolean isFullRebuild, boolean isNewCF) { String keyspaceName = baseCfs.keyspace.getName(); @@ -650,7 +696,10 @@ private synchronized void markIndexesBuilding(Set indexes, boolean isFull AtomicInteger counter = inProgressBuilds.computeIfAbsent(indexName, ignored -> new AtomicInteger(0)); if (isFullRebuild) + { needsFullRebuild.remove(indexName); + makeIndexNonQueryable(index, Index.Status.FULL_REBUILD_STARTED); + } if (counter.getAndIncrement() == 0 && DatabaseDescriptor.isDaemonInitialized() && !isNewCF) SystemKeyspace.setIndexRemoved(keyspaceName, indexName); @@ -668,14 +717,8 @@ private synchronized void markIndexBuilt(Index index, boolean isFullRebuild) { String indexName = index.getIndexMetadata().name; if (isFullRebuild) - { - if (queryableIndexes.add(indexName)) - logger.info("Index [{}] became queryable after successful build.", indexName); + makeIndexQueryable(index, Index.Status.BUILD_SUCCEEDED); - if (writableIndexes.put(indexName, index) == null) - logger.info("Index [{}] became writable after successful build.", indexName); - } - AtomicInteger counter = inProgressBuilds.get(indexName); if (counter != null) { @@ -742,6 +785,8 @@ private synchronized void markIndexRemoved(String indexName) writableIndexes.remove(indexName); needsFullRebuild.remove(indexName); inProgressBuilds.remove(indexName); + // remove existing indexing status + propagateLocalIndexStatus(keyspace.getName(), indexName, Index.Status.DROPPED); } public Index getIndexByName(String indexName) @@ -755,8 +800,10 @@ private Index createInstance(IndexMetadata indexDef) if (indexDef.isCustom()) { assert indexDef.options != null; - String className = indexDef.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME); + // Find any aliases to the fully qualified index class name: + String className = IndexMetadata.expandAliases(indexDef.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME)); assert !Strings.isNullOrEmpty(className); + try { Class indexClass = FBUtilities.classForName(className, "Index"); @@ -790,6 +837,8 @@ public void dropAllIndexes() { markAllIndexesRemoved(); invalidateAllIndexesBlocking(); + + indexGroups.forEach((key, group) -> group.invalidate()); } @VisibleForTesting @@ -851,7 +900,7 @@ private void flushIndexesBlocking(Set indexes, FutureCallback cal { indexes.forEach(index -> index.getBackingTable() - .map(cfs -> wait.add(cfs.forceFlush())) + .map(cfs -> wait.add(cfs.forceFlush(ColumnFamilyStore.FlushReason.INDEX_BUILD_COMPLETED))) .orElseGet(() -> nonCfsIndexes.add(index))); } @@ -892,7 +941,7 @@ public boolean hasIndexes() /** * When building an index against existing data in sstables, add the given partition to the index */ - public void indexPartition(DecoratedKey key, Set indexes, int pageSize) + public void indexPartition(DecoratedKey key, Set indexes, PageSize pageSize) { if (logger.isTraceEnabled()) logger.trace("Indexing partition {}", baseCfs.metadata().partitionKeyType.getString(key.getKey())); @@ -917,14 +966,16 @@ public void indexPartition(DecoratedKey key, Set indexes, int pageSize) try (UnfilteredRowIterator partition = page.next()) { - Set indexers = indexes.stream() - .map(index -> index.indexerFor(key, - partition.columns(), - nowInSec, - ctx, - IndexTransaction.Type.UPDATE)) - .filter(Objects::nonNull) - .collect(Collectors.toSet()); + Set indexers = indexGroups.values().stream() + .map(g -> g.indexerFor(indexes::contains, + key, + partition.columns(), + nowInSec, + ctx, + IndexTransaction.Type.UPDATE, + null)) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); // Short-circuit empty partitions if static row is processed or isn't read if (!readStatic && partition.isEmpty() && partition.staticRow().isEmpty()) @@ -980,39 +1031,12 @@ public void indexPartition(DecoratedKey key, Set indexes, int pageSize) /** * Return the page size used when indexing an entire partition */ - public int calculateIndexingPageSize() + public PageSize calculateIndexingPageSize() { if (Boolean.getBoolean("cassandra.force_default_indexing_page_size")) - return DEFAULT_PAGE_SIZE; - - double targetPageSizeInBytes = 32 * 1024 * 1024; - double meanPartitionSize = baseCfs.getMeanPartitionSize(); - if (meanPartitionSize <= 0) - return DEFAULT_PAGE_SIZE; - - int meanCellsPerPartition = baseCfs.getMeanEstimatedCellPerPartitionCount(); - if (meanCellsPerPartition <= 0) - return DEFAULT_PAGE_SIZE; - - int columnsPerRow = baseCfs.metadata().regularColumns().size(); - if (columnsPerRow <= 0) - return DEFAULT_PAGE_SIZE; - - int meanRowsPerPartition = meanCellsPerPartition / columnsPerRow; - double meanRowSize = meanPartitionSize / meanRowsPerPartition; - - int pageSize = (int) Math.max(1, Math.min(DEFAULT_PAGE_SIZE, targetPageSizeInBytes / meanRowSize)); + return PageSize.inRows(DEFAULT_PAGE_SIZE); - logger.trace("Calculated page size {} for indexing {}.{} ({}/{}/{}/{})", - pageSize, - baseCfs.metadata.keyspace, - baseCfs.metadata.name, - meanPartitionSize, - meanCellsPerPartition, - meanRowsPerPartition, - meanRowSize); - - return pageSize; + return PageSize.inBytes(32 * 1024 * 1024); } /** @@ -1023,6 +1047,9 @@ public int calculateIndexingPageSize() */ public void deletePartition(UnfilteredRowIterator partition, int nowInSec) { + if (!handles(IndexTransaction.Type.CLEANUP)) + return; + // we need to acquire memtable lock because secondary index deletion may // cause a race (see CASSANDRA-3712). This is done internally by the // index transaction when it commits @@ -1070,15 +1097,14 @@ public void deletePartition(UnfilteredRowIterator partition, int nowInSec) * ReadOrderGroup, or an estimate of the result size from an average index query. * * @param rowFilter RowFilter of the command to be executed - * @return an Index instance, ready to use during execution of the command, or null if none - * of the registered indexes can support the command. + * @return the best available index query plan for the row filter, or {@code null} if none of the registered indexes + * can support the command. */ - public Index getBestIndexFor(RowFilter rowFilter) + public Index.QueryPlan getBestIndexQueryPlanFor(RowFilter rowFilter) { if (indexes.isEmpty() || rowFilter.isEmpty()) return null; - Set searchableIndexes = new HashSet<>(); for (RowFilter.Expression expression : rowFilter) { if (expression.isCustom()) @@ -1088,46 +1114,63 @@ public Index getBestIndexFor(RowFilter rowFilter) RowFilter.CustomExpression customExpression = (RowFilter.CustomExpression) expression; logger.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name); Tracing.trace("Command contains a custom index expression, using target index {}", customExpression.getTargetIndex().name); - return indexes.get(customExpression.getTargetIndex().name); - } - else if (!expression.isUserDefined()) - { - indexes.values().stream() - .filter(index -> index.supportsExpression(expression.column(), expression.operator())) - .forEach(searchableIndexes::add); + Index.Group group = getIndexGroup(customExpression.getTargetIndex()); + return group == null ? null : group.queryPlanFor(rowFilter); } } - if (searchableIndexes.isEmpty()) + Set queryPlans = indexGroups.values() + .stream() + .map(g -> g.queryPlanFor(rowFilter)) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); + + if (queryPlans.isEmpty()) { logger.trace("No applicable indexes found"); Tracing.trace("No applicable indexes found"); return null; } - Index selected = searchableIndexes.size() == 1 - ? Iterables.getOnlyElement(searchableIndexes) - : searchableIndexes.stream() - .min((a, b) -> Longs.compare(a.getEstimatedResultRows(), - b.getEstimatedResultRows())) - .orElseThrow(() -> new AssertionError("Could not select most selective index")); + // find the best plan + Index.QueryPlan selected = queryPlans.size() == 1 + ? Iterables.getOnlyElement(queryPlans) + : queryPlans.stream() + .min(Comparator.naturalOrder()) + .orElseThrow(() -> new AssertionError("Could not select most selective index")); // pay for an additional threadlocal get() rather than build the strings unnecessarily if (Tracing.isTracing()) { Tracing.trace("Index mean cardinalities are {}. Scanning with {}.", - searchableIndexes.stream().map(i -> i.getIndexMetadata().name + ':' + i.getEstimatedResultRows()) - .collect(Collectors.joining(",")), - selected.getIndexMetadata().name); + queryPlans.stream() + .map(p -> commaSeparated(p.getIndexes()) + ':' + p.getEstimatedResultRows()) + .collect(Collectors.joining(",")), + commaSeparated(selected.getIndexes())); } return selected; } + private static String commaSeparated(Collection indexes) + { + return indexes.stream().map(i -> i.getIndexMetadata().name).collect(Collectors.joining(",")); + } + + public Optional getBestIndexFor(RowFilter.Expression expression) { return indexes.values().stream().filter((i) -> i.supportsExpression(expression.column(), expression.operator())).findFirst(); } + public Optional getBestIndexFor(RowFilter.Expression expression, Class indexType) + { + return indexes.values() + .stream() + .filter(i -> indexType.isInstance(i) && i.supportsExpression(expression.column(), expression.operator())) + .map(indexType::cast) + .findFirst(); + } + /** * Called at write time to ensure that values present in the update * are valid according to the rules of all registered indexes which @@ -1147,23 +1190,50 @@ public void validate(PartitionUpdate update) throws InvalidRequestException /* * IndexRegistry methods */ - - public void registerIndex(Index index) + public void registerIndex(Index index, Object groupKey, Supplier groupSupplier) { String name = index.getIndexMetadata().name; indexes.put(name, index); logger.trace("Registered index {}", name); - } - public void unregisterIndex(Index index) - { - unregisterIndex(index.getIndexMetadata().name); + // instantiate and add the index group if it hasn't been already added + Index.Group group = indexGroups.computeIfAbsent(groupKey, k -> groupSupplier.get()); + + // add the created index to its group if it is not a singleton group + if (!(group instanceof SingletonIndexGroup)) + { + if (index.getBackingTable().isPresent()) + throw new InvalidRequestException("Indexes belonging to a group of indexes shouldn't have a backing table"); + + group.addIndex(index); + } } private Index unregisterIndex(String name) { Index removed = indexes.remove(name); logger.trace(removed == null ? "Index {} was not registered" : "Removed index {} from registry", name); + + if (removed != null) + { + // Remove the index from any non-singleton groups... + for (Index.Group group : listIndexGroups()) + { + if (!(group instanceof SingletonIndexGroup) && group.containsIndex(removed)) + { + group.removeIndex(removed); + + if (group.getIndexes().isEmpty()) + { + indexGroups.remove(group); + } + } + } + + // ...and remove singleton groups entirely. + indexGroups.remove(removed); + } + return removed; } @@ -1177,6 +1247,42 @@ public Collection listIndexes() return ImmutableSet.copyOf(indexes.values()); } + public Set listIndexGroups() + { + return ImmutableSet.copyOf(indexGroups.values()); + } + + public Index.Group getIndexGroup(Object key) + { + return indexGroups.get(key); + } + + /** + * Returns the {@link Index.Group} the specified index belongs to, as specified during registering with + * {@link #registerIndex(Index, Object, Supplier)}. + * + * @param metadata the index metadata + * @return the group the index belongs to, or {@code null} if the index is not registered or if it hasn't been + * associated to any group + */ + @Nullable + public Index.Group getIndexGroup(IndexMetadata metadata) + { + Index index = getIndex(metadata); + return index == null ? null : getIndexGroup(index); + } + + @VisibleForTesting + public boolean needsFullRebuild(String index) + { + return needsFullRebuild.contains(index); + } + + public Index.Group getIndexGroup(Index index) + { + return indexGroups.values().stream().filter(g -> g.containsIndex(index)).findAny().orElse(null); + } + /* * Handling of index updates. * Implementations of the various IndexTransaction interfaces, for keeping indexes in sync with base data @@ -1186,23 +1292,28 @@ public Collection listIndexes() /** * Transaction for updates on the write path. */ - public UpdateTransaction newUpdateTransaction(PartitionUpdate update, WriteContext ctx, int nowInSec) + public UpdateTransaction newUpdateTransaction(PartitionUpdate update, WriteContext ctx, int nowInSec, Memtable memtable) { if (!hasIndexes()) return UpdateTransaction.NO_OP; - - ArrayList idxrs = new ArrayList<>(); - for (Index i : writableIndexes.values()) - { - Index.Indexer idxr = i.indexerFor(update.partitionKey(), update.columns(), nowInSec, ctx, IndexTransaction.Type.UPDATE); - if (idxr != null) - idxrs.add(idxr); - } - - if (idxrs.size() == 0) - return UpdateTransaction.NO_OP; - else - return new WriteTimeTransaction(idxrs.toArray(new Index.Indexer[idxrs.size()])); + + Index.Indexer[] indexers = listIndexGroups().stream() + .map(g -> g.indexerFor(writableIndexSelector(), + update.partitionKey(), + update.columns(), + nowInSec, + ctx, + IndexTransaction.Type.UPDATE, + memtable)) + .filter(Objects::nonNull) + .toArray(Index.Indexer[]::new); + + return indexers.length == 0 ? UpdateTransaction.NO_OP : new WriteTimeTransaction(indexers); + } + + private Predicate writableIndexSelector() + { + return index -> writableIndexes.containsKey(index.getIndexMetadata().name); } /** @@ -1214,7 +1325,7 @@ public CompactionTransaction newCompactionTransaction(DecoratedKey key, int nowInSec) { // the check for whether there are any registered indexes is already done in CompactionIterator - return new IndexGCTransaction(key, regularAndStaticColumns, keyspace, versions, nowInSec, writableIndexes.values()); + return new IndexGCTransaction(key, regularAndStaticColumns, keyspace, versions, nowInSec, listIndexGroups(), writableIndexSelector()); } /** @@ -1227,7 +1338,21 @@ public CleanupTransaction newCleanupTransaction(DecoratedKey key, if (!hasIndexes()) return CleanupTransaction.NO_OP; - return new CleanupGCTransaction(key, regularAndStaticColumns, keyspace, nowInSec, writableIndexes.values()); + return new CleanupGCTransaction(key, regularAndStaticColumns, keyspace, nowInSec, listIndexGroups(), writableIndexSelector()); + } + + /** + * @param type index transaction type + * @return true if at least one of the indexes will be able to handle given index transaction type + */ + public boolean handles(IndexTransaction.Type type) + { + for (Index.Group group : indexGroups.values()) + { + if (group.handles(type)) + return true; + } + return false; } /** @@ -1281,19 +1406,19 @@ public void onUpdated(Row existing, Row updated) // diff listener collates the columns to be added & removed from the indexes RowDiffListener diffListener = new RowDiffListener() { - public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original) + public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original) { } - public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original) + public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original) { } - public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original) + public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original) { } - public void onCell(int i, Clustering clustering, Cell merged, Cell original) + public void onCell(int i, Clustering clustering, Cell merged, Cell original) { if (merged != null && !merged.equals(original)) toInsert.addCell(merged); @@ -1315,7 +1440,7 @@ public void commit() indexer.finish(); } - private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell) + private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell) { // If either the value or timestamp is different, then we // should delete from the index. If not, then we can infer that @@ -1326,7 +1451,7 @@ private boolean shouldCleanupOldValue(Cell oldCell, Cell newCel // Completely identical cells (including expiring columns with // identical ttl & localExpirationTime) will not get this far due // to the oldCell.equals(newCell) in StandardUpdater.update - return !Cells.valueEqual(oldCell, newCell) || oldCell.timestamp() != newCell.timestamp(); + return !oldCell.value().equals(newCell.value()) || oldCell.timestamp() != newCell.timestamp(); } } @@ -1343,7 +1468,8 @@ private static final class IndexGCTransaction implements CompactionTransaction private final Keyspace keyspace; private final int versions; private final int nowInSec; - private final Collection indexes; + private final Collection indexGroups; + private final Predicate writableIndexSelector; private Row[] rows; @@ -1352,14 +1478,16 @@ private IndexGCTransaction(DecoratedKey key, Keyspace keyspace, int versions, int nowInSec, - Collection indexes) + Collection indexGroups, + Predicate writableIndexSelector) { this.key = key; this.columns = columns; this.keyspace = keyspace; this.versions = versions; - this.indexes = indexes; + this.indexGroups = indexGroups; this.nowInSec = nowInSec; + this.writableIndexSelector = writableIndexSelector; } public void start() @@ -1375,27 +1503,27 @@ public void onRowMerge(Row merged, Row... versions) final Row.Builder[] builders = new Row.Builder[versions.length]; RowDiffListener diffListener = new RowDiffListener() { - public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original) + public void onPrimaryKeyLivenessInfo(int i, Clustering clustering, LivenessInfo merged, LivenessInfo original) { if (original != null && (merged == null || !merged.isLive(nowInSec))) getBuilder(i, clustering).addPrimaryKeyLivenessInfo(original); } - public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original) + public void onDeletion(int i, Clustering clustering, Row.Deletion merged, Row.Deletion original) { } - public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original) + public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original) { } - public void onCell(int i, Clustering clustering, Cell merged, Cell original) + public void onCell(int i, Clustering clustering, Cell merged, Cell original) { if (original != null && (merged == null || !merged.isLive(nowInSec))) getBuilder(i, clustering).addCell(original); } - private Row.Builder getBuilder(int index, Clustering clustering) + private Row.Builder getBuilder(int index, Clustering clustering) { if (builders[index] == null) { @@ -1420,9 +1548,9 @@ public void commit() try (WriteContext ctx = keyspace.getWriteHandler().createContextForIndexing()) { - for (Index index : indexes) + for (Index.Group group : indexGroups) { - Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, ctx, Type.COMPACTION); + Index.Indexer indexer = group.indexerFor(writableIndexSelector, key, columns, nowInSec, ctx, Type.COMPACTION, null); if (indexer == null) continue; @@ -1448,7 +1576,8 @@ private static final class CleanupGCTransaction implements CleanupTransaction private final RegularAndStaticColumns columns; private final Keyspace keyspace; private final int nowInSec; - private final Collection indexes; + private final Collection indexGroups; + private final Predicate writableIndexSelector; private Row row; private DeletionTime partitionDelete; @@ -1457,13 +1586,15 @@ private CleanupGCTransaction(DecoratedKey key, RegularAndStaticColumns columns, Keyspace keyspace, int nowInSec, - Collection indexes) + Collection indexGroups, + Predicate writableIndexSelector) { this.key = key; this.columns = columns; this.keyspace = keyspace; - this.indexes = indexes; + this.indexGroups = indexGroups; this.nowInSec = nowInSec; + this.writableIndexSelector = writableIndexSelector; } public void start() @@ -1487,9 +1618,9 @@ public void commit() try (WriteContext ctx = keyspace.getWriteHandler().createContextForIndexing()) { - for (Index index : indexes) + for (Index.Group group : indexGroups) { - Index.Indexer indexer = index.indexerFor(key, columns, nowInSec, ctx, Type.CLEANUP); + Index.Indexer indexer = group.indexerFor(writableIndexSelector, key, columns, nowInSec, ctx, Type.CLEANUP, null); if (indexer == null) continue; @@ -1562,4 +1693,140 @@ public static void shutdownAndWait(long timeout, TimeUnit units) throws Interrup shutdown(asyncExecutor, blockingExecutor); awaitTermination(timeout, units, asyncExecutor, blockingExecutor); } + + /** + * Remove endpoints whose indexes are not queryable for the specified {@link Index.QueryPlan}. + * + * @param liveEndpoints current live endpoints where non-queryable endpoints will be removed + * @param keyspace to be queried + * @param indexQueryPlan index query plan used in the read command + * @param level consistency level of read command + */ + public static > E filterForQuery(E liveEndpoints, Keyspace keyspace, Index.QueryPlan indexQueryPlan, ConsistencyLevel level) + { + E queryableEndpoints = liveEndpoints.filter(replica -> { + + for (Index index : indexQueryPlan.getIndexes()) + { + Index.Status status = getIndexStatus(replica.endpoint(), keyspace.getName(), index.getIndexMetadata().name); + if (!index.isQueryable(status)) + return false; + } + + return true; + }); + + int initial = liveEndpoints.size(); + int filtered = queryableEndpoints.size(); + + // Throw ReadFailureException if read request cannot satisfy Consistency Level due to non-queryable indexes. + // It is to provide a better UX, compared to throwing UnavailableException when the nodes are actually alive. + if (initial != filtered) + { + int required = level.blockFor(keyspace.getReplicationStrategy()); + if (required <= initial && required > filtered) + { + Map failureReasons = new HashMap<>(); + liveEndpoints.without(queryableEndpoints.endpoints()) + .forEach(replica -> failureReasons.put(replica.endpoint(), RequestFailureReason.INDEX_NOT_AVAILABLE)); + + throw new ReadFailureException(level, filtered, required, false, failureReasons); + } + } + + return queryableEndpoints; + } + + public void makeIndexNonQueryable(Index index, Index.Status status) + { + String name = index.getIndexMetadata().name; + if (indexes.get(name) == index) + { + propagateLocalIndexStatus(keyspace.getName(), name, status); + if (!index.isQueryable(status)) + queryableIndexes.remove(name); + } + } + + public void makeIndexQueryable(Index index, Index.Status status) + { + String name = index.getIndexMetadata().name; + if (indexes.get(name) == index) + { + propagateLocalIndexStatus(keyspace.getName(), name, status); + if (index.isQueryable(status)) + { + if (queryableIndexes.add(name)) + logger.info("Index [{}] became queryable after successful build.", name); + } + + if (writableIndexes.put(name, index) == null) + logger.info("Index [{}] became writable after successful build.", name); + } + } + + public static Index.Status getIndexStatus(InetAddressAndPort peer, String keyspace, String index) + { + return peerIndexStatus.getOrDefault(peer, Collections.emptyMap()) + .getOrDefault(identifier(keyspace, index), Index.Status.UNKNOWN); + } + + public synchronized static void receivePeerIndexStatus(InetAddressAndPort endpoint, VersionedValue versionedValue) + { + try + { + if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) + return; + + Map peerStatus = (Map) JSONValue.parseWithException(versionedValue.value); + Map indexStatus = new ConcurrentHashMap<>(); + + for (Map.Entry e : peerStatus.entrySet()) + { + String keyspaceIndex = e.getKey(); + Index.Status status = Index.Status.valueOf(e.getValue()); + indexStatus.put(keyspaceIndex, status); + } + + peerIndexStatus.put(endpoint, indexStatus); + } + catch (Throwable e) + { + logger.warn("Unable to parse index status: {}", e.getMessage()); + } + } + + @VisibleForTesting + public synchronized static void propagateLocalIndexStatus(String keyspace, String index, Index.Status status) + { + try + { + Map states = peerIndexStatus.computeIfAbsent(FBUtilities.getBroadcastAddressAndPort(), + k -> new ConcurrentHashMap<>()); + String keyspaceIndex = identifier(keyspace, index); + + if (status == Index.Status.DROPPED) + states.remove(keyspaceIndex); + else + states.put(keyspaceIndex, status); + + String newStatus = JSONValue.toJSONString(states.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().toString()))); + statusPropagationExecutor.submit(() -> { + // schedule gossiper update asynchronously to avoid potential deadlock when another thread is holding + // gossiper taskLock. + VersionedValue value = StorageService.instance.valueFactory.indexStatus(newStatus); + Gossiper.instance.addLocalApplicationState(ApplicationState.INDEX_STATUS, value); + }); + } + catch (Throwable e) + { + logger.warn("Unable to propagate index status: {}", e.getMessage()); + } + } + + private static String identifier(String keyspace, String index) + { + return new StringBuilder().append(keyspace).append('.').append(index).toString(); + } } diff --git a/src/java/org/apache/cassandra/index/SingletonIndexGroup.java b/src/java/org/apache/cassandra/index/SingletonIndexGroup.java new file mode 100644 index 000000000000..c70b996f4ca8 --- /dev/null +++ b/src/java/org/apache/cassandra/index/SingletonIndexGroup.java @@ -0,0 +1,112 @@ +/* + * + * 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.index; + +import java.util.Collections; +import java.util.Set; +import java.util.function.Predicate; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.WriteContext; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.index.transactions.IndexTransaction; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; +import org.apache.cassandra.schema.TableMetadata; + +/** + * An {@link Index.Group} containing a single {@link Index}, to which it just delegates the calls. + */ +public class SingletonIndexGroup implements Index.Group +{ + private final Index delegate; + private final Set indexes; + + protected SingletonIndexGroup(Index delegate) + { + this.delegate = delegate; + this.indexes = Collections.singleton(delegate); + } + + @Override + public Set getIndexes() + { + return indexes; + } + + public Index getIndex() + { + return delegate; + } + + @Override + public void addIndex(Index index) + { + throw new UnsupportedOperationException(); + } + + @Override + public void removeIndex(Index index) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean containsIndex(Index index) + { + return index.equals(delegate); + } + + @Override + public Index.Indexer indexerFor(Predicate indexSelector, + DecoratedKey key, + RegularAndStaticColumns columns, + int nowInSec, + WriteContext ctx, + IndexTransaction.Type transactionType, + Memtable memtable) + { + return indexSelector.test(delegate) ? delegate.indexerFor(key, columns, nowInSec, ctx, transactionType, memtable) : null; + } + + @Override + public Index.QueryPlan queryPlanFor(RowFilter rowFilter) + { + return SingletonIndexQueryPlan.create(delegate, rowFilter); + } + + @Override + public SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker, TableMetadata tableMetadata) + { + return delegate.getFlushObserver(descriptor, tracker); + } + + @Override + public Set getComponents() + { + return delegate.getComponents(); + } +} diff --git a/src/java/org/apache/cassandra/index/SingletonIndexQueryPlan.java b/src/java/org/apache/cassandra/index/SingletonIndexQueryPlan.java new file mode 100644 index 000000000000..b475cee145a3 --- /dev/null +++ b/src/java/org/apache/cassandra/index/SingletonIndexQueryPlan.java @@ -0,0 +1,87 @@ +/* + * + * 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.index; + +import java.util.Collections; +import java.util.Set; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.filter.RowFilter; + +public class SingletonIndexQueryPlan implements Index.QueryPlan +{ + private final Index index; + private final Set indexes; + private final RowFilter postIndexFilter; + + protected SingletonIndexQueryPlan(Index index, RowFilter postIndexFilter) + { + this.index = index; + this.indexes = Collections.singleton(index); + this.postIndexFilter = postIndexFilter; + } + + @Nullable + protected static SingletonIndexQueryPlan create(Index index, RowFilter rowFilter) + { + for (RowFilter.Expression e : rowFilter.getExpressions()) + { + if (index.supportsExpression(e.column(), e.operator())) + return new SingletonIndexQueryPlan(index, index.getPostIndexQueryFilter(rowFilter)); + } + + return null; + } + + @Override + public Set getIndexes() + { + return indexes; + } + + @Override + @Nonnull + public Index getFirst() + { + return index; + } + + @Override + public long getEstimatedResultRows() + { + return index.getEstimatedResultRows(); + } + + @Override + public Index.Searcher searcherFor(ReadCommand command) + { + return index.searcherFor(command); + } + + @Override + public RowFilter postIndexQueryFilter() + { + return postIndexFilter; + } +} diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java index ea5f8e2f613c..ae5ae7a778d1 100644 --- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java +++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java @@ -33,6 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.schema.ColumnMetadata; @@ -184,7 +185,7 @@ public Optional getBackingTable() public Callable getBlockingFlushTask() { return () -> { - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.INDEX_TABLE_FLUSH); return null; }; } @@ -271,14 +272,6 @@ public long getEstimatedResultRows() return indexCfs.getMeanRowCount(); } - /** - * No post processing of query results, just return them unchanged - */ - public BiFunction postProcessorFor(ReadCommand command) - { - return (partitionIterator, readCommand) -> partitionIterator; - } - public RowFilter getPostIndexQueryFilter(RowFilter filter) { return getTargetExpression(filter.getExpressions()).map(filter::without) @@ -340,7 +333,8 @@ public Indexer indexerFor(final DecoratedKey key, final RegularAndStaticColumns columns, final int nowInSec, final WriteContext ctx, - final IndexTransaction.Type transactionType) + final IndexTransaction.Type transactionType, + final Memtable memtable) { /** * Indexes on regular and static columns (the non primary-key ones) only care about updates with live @@ -527,7 +521,7 @@ private void insert(ByteBuffer rowKey, cell)); Row row = BTreeRow.noCellLiveRow(buildIndexClustering(rowKey, clustering, cell), info); PartitionUpdate upd = partitionUpdate(valueKey, row); - indexCfs.getWriteHandler().write(upd, ctx, UpdateTransaction.NO_OP); + indexCfs.getWriteHandler().write(upd, ctx, false); logger.trace("Inserted entry into index for value {}", valueKey); } @@ -573,7 +567,7 @@ private void doDelete(DecoratedKey indexKey, { Row row = BTreeRow.emptyDeletedRow(indexClustering, Row.Deletion.regular(deletion)); PartitionUpdate upd = partitionUpdate(indexKey, row); - indexCfs.getWriteHandler().write(upd, ctx, UpdateTransaction.NO_OP); + indexCfs.getWriteHandler().write(upd, ctx, false); logger.trace("Removed index entry for value {}", indexKey); } @@ -662,7 +656,7 @@ private void invalidate() CompactionManager.instance.interruptCompactionForCFs(cfss, (sstable) -> true, true); CompactionManager.instance.waitForCessation(cfss, (sstable) -> true); Keyspace.writeOrder.awaitNewBarrier(); - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.INDEX_REMOVED); indexCfs.readOrdering.awaitNewBarrier(); indexCfs.invalidate(); } @@ -688,7 +682,7 @@ private Callable getBuildIndexTask() @SuppressWarnings("resource") private void buildBlocking() { - baseCfs.forceBlockingFlush(); + baseCfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.INDEX_BUILD_STARTED); try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL)); Refs sstables = viewFragment.refs) @@ -712,7 +706,7 @@ private void buildBlocking() ImmutableSet.copyOf(sstables)); Future future = CompactionManager.instance.submitIndexBuild(builder); FBUtilities.waitOnFuture(future); - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.INDEX_BUILD_COMPLETED); } logger.info("Index build of {} complete", metadata.name); } @@ -742,7 +736,7 @@ public static TableMetadata indexCfsMetadata(TableMetadata baseCfsMetadata, Inde TableMetadata.builder(baseCfsMetadata.keyspace, baseCfsMetadata.indexTableName(indexMetadata), baseCfsMetadata.id) .kind(TableMetadata.Kind.INDEX) .partitioner(new LocalPartitioner(indexedValueType)) - .addPartitionKeyColumn(indexedColumn.name, indexedColumn.type) + .addPartitionKeyColumn(indexedColumn.name, indexedValueType) .addClusteringColumn("partition_key", baseCfsMetadata.partitioner.partitionOrdering()); // Adding clustering columns, which depends on the index type. diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java b/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java index e2e0600aaeb8..698264688bec 100644 --- a/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java +++ b/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java @@ -54,6 +54,12 @@ public CassandraIndexSearcher(ReadCommand command, this.index = index; } + @Override + public ReadCommand command() + { + return command; + } + @SuppressWarnings("resource") // Both the OpOrder and 'indexIter' are closed on exception, or through the closing of the result // of this method. public UnfilteredPartitionIterator search(ReadExecutionController executionController) diff --git a/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java b/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java index 3c005c42525b..5e6af413d899 100644 --- a/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java +++ b/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java @@ -21,9 +21,9 @@ import java.util.Set; import java.util.UUID; +import org.apache.cassandra.cql3.PageSize; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.compaction.CompactionInfo; import org.apache.cassandra.db.compaction.CompactionInterruptedException; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.index.Index; @@ -52,25 +52,25 @@ public CollatedViewIndexBuilder(ColumnFamilyStore cfs, Set indexers, Redu this.sstables = sstables; } - public CompactionInfo getCompactionInfo() + public OperationProgress getProgress() { - return new CompactionInfo(cfs.metadata(), - OperationType.INDEX_BUILD, - iter.getBytesRead(), - iter.getTotalBytes(), - compactionId, - sstables); + return new OperationProgress(cfs.metadata(), + OperationType.INDEX_BUILD, + iter.getBytesRead(), + iter.getTotalBytes(), + compactionId, + sstables); } public void build() { try { - int pageSize = cfs.indexManager.calculateIndexingPageSize(); + PageSize pageSize = cfs.indexManager.calculateIndexingPageSize(); while (iter.hasNext()) { if (isStopRequested()) - throw new CompactionInterruptedException(getCompactionInfo()); + throw new CompactionInterruptedException(getProgress()); DecoratedKey key = iter.next(); cfs.indexManager.indexPartition(key, indexers, pageSize); } diff --git a/src/java/org/apache/cassandra/index/sai/ColumnContext.java b/src/java/org/apache/cassandra/index/sai/ColumnContext.java new file mode 100644 index 000000000000..99221b58540b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/ColumnContext.java @@ -0,0 +1,594 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.statements.schema.IndexTarget; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.ComplexColumnData; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.index.TargetParser; +import org.apache.cassandra.index.sai.analyzer.AbstractAnalyzer; +import org.apache.cassandra.index.sai.disk.IndexSearcher; +import org.apache.cassandra.index.sai.disk.IndexWriterConfig; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.memory.MemtableIndex; +import org.apache.cassandra.index.sai.metrics.ColumnQueryMetrics; +import org.apache.cassandra.index.sai.metrics.IndexMetrics; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.RangeUnionIterator; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.index.sai.view.IndexViewManager; +import org.apache.cassandra.index.sai.view.View; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.IndexMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Pair; + +/** + * Manage metadata for each column index. + */ +public class ColumnContext +{ + private static final Logger logger = LoggerFactory.getLogger(ColumnContext.class); + + private static final Set> EQ_ONLY_TYPES = + ImmutableSet.of(UTF8Type.instance, AsciiType.instance, BooleanType.instance, UUIDType.instance); + + private final AbstractType partitionKeyType; + private final ClusteringComparator clusteringComparator; + + private final String keyspace; + private final String table; + private final Pair target; + private final AbstractType validator; + + // Config can be null if the column context is "fake" (i.e. created for a filtering expression). + private final IndexMetadata config; + + private final ConcurrentMap liveMemtables = new ConcurrentHashMap<>(); + + private final IndexViewManager viewManager; + private final IndexMetrics indexMetrics; + private final ColumnQueryMetrics columnQueryMetrics; + private final IndexWriterConfig indexWriterConfig; + + public ColumnContext(TableMetadata tableMeta, IndexMetadata metadata) + { + this.keyspace = tableMeta.keyspace; + this.table = tableMeta.name; + this.partitionKeyType = tableMeta.partitionKeyType; + this.clusteringComparator = tableMeta.comparator; + this.target = TargetParser.parse(tableMeta, metadata); + this.config = metadata; + this.viewManager = new IndexViewManager(this); + this.indexMetrics = new IndexMetrics(this, tableMeta); + this.validator = TypeUtil.cellValueType(target); + + String fullIndexName = String.format("%s.%s.%s", this.keyspace, this.table, this.config.name); + this.indexWriterConfig = IndexWriterConfig.fromOptions(fullIndexName, validator, config.options); + this.columnQueryMetrics = isLiteral() ? new ColumnQueryMetrics.TrieIndexMetrics(getIndexName(), tableMeta) + : new ColumnQueryMetrics.BKDIndexMetrics(getIndexName(), tableMeta); + + logger.info(logMessage("Initialized column context with index writer config: {}"), + this.indexWriterConfig.toString()); + } + + @VisibleForTesting + public ColumnContext(String keyspace, + String table, + AbstractType partitionKeyType, + ClusteringComparator clusteringComparator, + ColumnMetadata column, + IndexMetadata config, + IndexWriterConfig indexWriterConfig) + { + this.keyspace = keyspace; + this.table = table; + this.partitionKeyType = partitionKeyType; + this.clusteringComparator = clusteringComparator; + this.target = Pair.create(column, IndexTarget.Type.SIMPLE); + this.validator = column.type; + this.config = config; + this.viewManager = null; + this.indexMetrics = null; + this.columnQueryMetrics = null; + this.indexWriterConfig = indexWriterConfig; + } + + public ColumnContext(TableMetadata table, ColumnMetadata column) + { + this.keyspace = table.keyspace; + this.table = table.name; + this.partitionKeyType = table.partitionKeyType; + this.clusteringComparator = table.comparator; + this.target = TargetParser.parse(table, column.name.toString()); + this.validator = target == null ? null : TypeUtil.cellValueType(target); + this.config = null; + this.viewManager = null; + this.indexMetrics = null; + this.columnQueryMetrics = null; + this.indexWriterConfig = IndexWriterConfig.emptyConfig(); + } + + public AbstractType keyValidator() + { + return partitionKeyType; + } + + public ClusteringComparator clusteringComparator() + { + return clusteringComparator; + } + + public IndexMetrics getIndexMetrics() + { + return indexMetrics; + } + + public ColumnQueryMetrics getColumnQueryMetrics() + { + return columnQueryMetrics; + } + + public String getTable() + { + return table; + } + + public long index(DecoratedKey key, Row row, Memtable mt) + { + MemtableIndex current = liveMemtables.get(mt); + + // We expect the relevant IndexMemtable to be present most of the time, so only make the + // call to computeIfAbsent() if it's not. (see https://bugs.openjdk.java.net/browse/JDK-8161372) + MemtableIndex target = (current != null) + ? current + : liveMemtables.computeIfAbsent(mt, memtable -> new MemtableIndex(this, mt)); + + long start = System.nanoTime(); + + long bytes = 0; + + if (isNonFrozenCollection()) + { + Iterator bufferIterator = getValuesOf(row, FBUtilities.nowInSeconds()); + if (bufferIterator != null) + { + while (bufferIterator.hasNext()) + { + ByteBuffer value = bufferIterator.next(); + bytes += target.index(key, row.clustering(), value); + } + } + } + else + { + ByteBuffer value = getValueOf(key, row, FBUtilities.nowInSeconds()); + target.index(key, row.clustering(), value); + } + indexMetrics.memtableIndexWriteLatency.update(System.nanoTime() - start, TimeUnit.NANOSECONDS); + return bytes; + } + + public void renewMemtable(Memtable renewed) + { + for (Memtable memtable : liveMemtables.keySet()) + { + // remove every index but the one that corresponds to the post-truncate Memtable + if (renewed != memtable) + { + liveMemtables.remove(memtable); + } + } + } + + public void discardMemtable(Memtable discarded) + { + liveMemtables.remove(discarded); + } + + public MemtableIndex getPendingMemtableIndex(LifecycleNewTracker tracker) + { + return liveMemtables.keySet().stream() + .filter(m -> tracker.equals(m.getFlushTransaction())) + .findFirst() + .map(liveMemtables::get) + .orElse(null); + } + + public RangeIterator searchMemtable(Expression e, AbstractBounds keyRange) + { + Collection memtables = liveMemtables.values(); + + if (memtables.isEmpty()) + { + return RangeIterator.empty(); + } + + RangeUnionIterator.Builder builder = RangeUnionIterator.builder(); + + for (MemtableIndex index : memtables) + { + builder.add(index.search(e, keyRange)); + } + + return builder.build(); + } + + public long liveMemtableWriteCount() + { + return liveMemtables.values().stream().mapToLong(MemtableIndex::writeCount).sum(); + } + + public long estimatedMemIndexMemoryUsed() + { + return liveMemtables.values().stream().mapToLong(MemtableIndex::estimatedMemoryUsed).sum(); + } + + /** + * @return A set of SSTables which have attached to them invalid index components. + */ + public Set onSSTableChanged(Collection oldSSTables, Collection newSSTables, boolean validate, boolean rename) + { + return viewManager.update(oldSSTables, newSSTables, validate, rename); + } + + public ColumnMetadata getDefinition() + { + return target.left; + } + + public AbstractType getValidator() + { + return validator; + } + + public boolean isNonFrozenCollection() + { + return TypeUtil.isNonFrozenCollection(target.left.type); + } + + public boolean isFrozen() + { + return TypeUtil.isFrozen(target.left.type); + } + + public String getColumnName() + { + return target.left.name.toString(); + } + + public String getIndexName() + { + return this.config == null ? null : config.name; + } + + public AbstractAnalyzer getAnalyzer() + { + Map options = config != null ? config.options : Collections.emptyMap(); + return AbstractAnalyzer.fromOptions(getValidator(), options); + } + + public IndexWriterConfig getIndexWriterConfig() + { + return indexWriterConfig; + } + + public View getView() + { + return viewManager.getView(); + } + + /** + * @return total number of per-index open files + */ + public int openPerIndexFiles() + { + return viewManager.getView().size() * IndexSearcher.openPerIndexFiles(getValidator()); + } + + public void drop(Collection sstablesToRebuild) + { + viewManager.drop(sstablesToRebuild); + } + + public boolean isIndexed() + { + return config != null; + } + + /** + * Called when index is dropped. Mark all {@link SSTableIndex} as obsolete and per-column index files + * will be removed when in-flight queries completed. + */ + public void invalidate() + { + liveMemtables.clear(); + viewManager.invalidate(); + indexMetrics.release(); + columnQueryMetrics.release(); + } + + @VisibleForTesting + public ConcurrentMap getLiveMemtables() + { + return liveMemtables; + } + + public boolean supports(Operator op) + { + if (op.isLike() || op == Operator.LIKE) return false; + + Expression.Op operator = Expression.Op.valueOf(op); + IndexTarget.Type type = target.right; + + if (isNonFrozenCollection()) + { + if (type == IndexTarget.Type.KEYS) return operator == Expression.Op.CONTAINS_KEY; + if (type == IndexTarget.Type.VALUES) return operator == Expression.Op.CONTAINS_VALUE; + return type == IndexTarget.Type.KEYS_AND_VALUES && operator == Expression.Op.EQ; + } + + if (type == IndexTarget.Type.FULL) + return operator == Expression.Op.EQ; + + AbstractType validator = getValidator(); + + if (operator == Expression.Op.IN) + return true; + + if (operator != Expression.Op.EQ && EQ_ONLY_TYPES.contains(validator)) return false; + + // RANGE only applicable to non-literal indexes + return (operator != null) && !(TypeUtil.isLiteral(validator) && operator == Expression.Op.RANGE); + } + + public ByteBuffer getValueOf(DecoratedKey key, Row row, int nowInSecs) + { + if (row == null) + return null; + + switch (target.left.kind) + { + case PARTITION_KEY: + return partitionKeyType instanceof CompositeType + ? CompositeType.extractComponent(key.getKey(), target.left.position()) + : key.getKey(); + case CLUSTERING: + // skip indexing of static clustering when regular column is indexed + return row.isStatic() ? null : row.clustering().bufferAt(target.left.position()); + + // treat static cell retrieval the same was as regular + // only if row kind is STATIC otherwise return null + case STATIC: + if (!row.isStatic()) + return null; + case REGULAR: + Cell cell = row.getCell(target.left); + return cell == null || !cell.isLive(nowInSecs) ? null : cell.buffer(); + + default: + return null; + } + } + + public Iterator getValuesOf(Row row, int nowInSecs) + { + if (row == null) + return null; + + switch (target.left.kind) + { + // treat static cell retrieval the same was as regular + // only if row kind is STATIC otherwise return null + case STATIC: + if (!row.isStatic()) + return null; + case REGULAR: + return TypeUtil.collectionIterator(validator, (ComplexColumnData)row.getComplexColumnData(target.left), target, nowInSecs); + + default: + return null; + } + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("columnName", getColumnName()) + .add("indexName", getIndexName()) + .toString(); + } + + public boolean isLiteral() + { + return TypeUtil.isLiteral(getValidator()); + } + + public boolean equals(Object obj) + { + if (obj == this) + return true; + + if (!(obj instanceof ColumnContext)) + return false; + + ColumnContext other = (ColumnContext) obj; + + return Objects.equals(target, other.target) && + Objects.equals(config, other.config) && + Objects.equals(partitionKeyType, other.partitionKeyType) && + Objects.equals(clusteringComparator, other.clusteringComparator); + } + + public int hashCode() + { + return Objects.hash(target, config, partitionKeyType, clusteringComparator); + } + + /** + * A helper method for constructing consistent log messages for specific column indexes. + * + * Example: For the index "idx" in keyspace "ks" on table "tb", calling this method with the raw message + * "Flushing new index segment..." will produce... + * + * "[ks.tb.idx] Flushing new index segment..." + * + * @param message The raw content of a logging message, without information identifying it with an index. + * + * @return A log message with the proper keyspace, table and index name prepended to it. + */ + public String logMessage(String message) + { + // Index names are unique only within a keyspace. + return String.format("[%s.%s.%s] %s", keyspace, table, config == null ? "?" : config.name, message); + } + + /** + * @return the indexes that are built on the given SSTables on the left and corrupted indexes' + * corresponding contexts on the right + */ + public Pair, Set> getBuiltIndexes(Collection sstableContexts, boolean validate, boolean rename) + { + Set valid = new HashSet<>(sstableContexts.size()); + Set invalid = new HashSet<>(); + + for (SSTableContext context : sstableContexts) + { + if (context.sstable.isMarkedCompacted()) + continue; + + if (!IndexComponents.isColumnIndexComplete(context.descriptor(), getIndexName())) + { + logger.debug(logMessage("An on-disk index build for SSTable {} has not completed."), context.descriptor()); + continue; + } + + if (IndexComponents.isColumnIndexEmpty(context.descriptor(), getIndexName())) + { + logger.debug(logMessage("No on-disk index was built for SSTable {} because the SSTable " + + "had no indexable rows for the index."), context.descriptor()); + continue; + } + + // TODO: does the column name need to be encoded since it's being included in a filename? + final IndexComponents components = IndexComponents.create(getIndexName(), context.sstable()); + + try + { + if (validate) + { + components.validatePerColumnComponents(isLiteral()); + } + + SSTableIndex index = new SSTableIndex(context, this, components); + logger.debug(logMessage("Successfully created index for SSTable {}."), context.descriptor()); + + // Try to add new index to the set, if set already has such index, we'll simply release and move on. + // This covers situation when SSTable collection has the same SSTable multiple + // times because we don't know what kind of collection it actually is. + if (!valid.add(index)) + { + index.release(); + } + } + catch (Throwable e) + { + invalid.add(context); + logger.warn(logMessage("Invalid per-column component for SSTable {}"), context.descriptor(), e); + } + } + + return Pair.create(valid, invalid); + } + + /** + * @return the number of indexed rows in this index (aka. pair of term and rowId) + */ + public long getCellCount() + { + return getView().getIndexes() + .stream() + .mapToLong(SSTableIndex::getRowCount) + .sum(); + } + + /** + * @return the total size (in bytes) of per-column index components + */ + public long diskUsage() + { + return getView().getIndexes() + .stream() + .mapToLong(SSTableIndex::sizeOfPerColumnComponents) + .sum(); + } + + /** + * @return the total memory usage (in bytes) of per-column index on-disk data structure + */ + public long indexFileCacheSize() + { + return getView().getIndexes() + .stream() + .mapToLong(SSTableIndex::indexFileCacheSize) + .sum(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/QueryContext.java b/src/java/org/apache/cassandra/index/sai/QueryContext.java new file mode 100644 index 000000000000..a5abf2a78d82 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/QueryContext.java @@ -0,0 +1,102 @@ +/* + * 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.index.sai; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.index.sai.utils.AbortedOperationException; +import org.apache.cassandra.io.sstable.format.SSTableReader; + +/** + * Tracks state relevant to the execution of a single query, including metrics and timeout monitoring. + * + * Fields here are non-volatile, as they are accessed from a single thread. + */ +@NotThreadSafe +public class QueryContext +{ + private final long queryStartTimeNanos; + + public final long executionQuotaNano; + + public long sstablesHit = 0; + public long segmentsHit = 0; + public long partitionsRead = 0; + public long rowsFiltered = 0; + + public long trieSegmentsHit = 0; + + public long bkdPostingListsHit = 0; + public long bkdSegmentsHit = 0; + + public long bkdPostingsSkips = 0; + public long bkdPostingsDecodes = 0; + + public long triePostingsSkips = 0; + public long triePostingsDecodes = 0; + + public long tokenSkippingCacheHits = 0; + public long tokenSkippingLookups = 0; + + public long queryTimeouts = 0; + + private final Map sstableQueryContexts = new HashMap<>(); + + @VisibleForTesting + public QueryContext() + { + this(DatabaseDescriptor.getRangeRpcTimeout(TimeUnit.MILLISECONDS)); + } + + public QueryContext(long executionQuotaMs) + { + this.executionQuotaNano = TimeUnit.MILLISECONDS.toNanos(executionQuotaMs); + queryStartTimeNanos = System.nanoTime(); + } + + public long totalQueryTimeNs() + { + return System.nanoTime() - queryStartTimeNanos; + } + + public void incSstablesHit() + { + sstablesHit++; + } + + public SSTableQueryContext getSSTableQueryContext(SSTableReader reader) + { + return sstableQueryContexts.computeIfAbsent(reader, k -> new SSTableQueryContext(this)); + } + + public void checkpoint() + { + if (totalQueryTimeNs() >= executionQuotaNano) + { + queryTimeouts++; + throw new AbortedOperationException(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/README.md b/src/java/org/apache/cassandra/index/sai/README.md new file mode 100644 index 000000000000..a475b0c2222e --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/README.md @@ -0,0 +1,129 @@ + + +# Storage-Attached Indexing + +## Overview +Storage-attached indexes are a new column-based secondary indexing apparatus for DSE. + +The project was inspired by OSS SASI (SSTable-Attached Secondary Indexes) and retains some of its high-level +architectural character (and even some actual code), but makes significant improvements in a number of areas: + +- The on-disk/SSTable index formats for both string and numeric data have been completely replaced. Strings are indexed + on disk using our proprietary on-disk byte-ordered trie data structure, while numeric types are indexed using Lucene's + balanced kd-tree. +- While indexes continue to be managed at the column level from the user's perspective, the storage design at the column + index level is row-based, with related offset and token information stored only once at the SSTable level. This + drastically reduces our on-disk footprint when several columns are indexed on the same table. +- The query path is synchronous and index searches run on IO threads. +- Tracing, metrics, virtual table-based metadata, RLAC, and snapshot-based backup/restore are supported out of the box. + +Many similarities with standard secondary indexes remain: + +- The full set of C* consistency levels is supported for both reads and writes. +- Index updates are synchronous with mutations and do not require any kind of read-before-write. +- Queries are implemented on the back of C* range reads. +- Paging is supported. +- Only token ordering of results is supported. +- Index builds are visible to operators as compactions and are executed on compaction threads. +- All DML and DDL statements are CQL-based. +- Single-node management operations are available via nodetool. (ex. stop & rebuild_index) + +## Quick Start + +The following short tutorial will get you up-and-running with storage-attached indexing. + +### Build and Start DSE + +1.) Make sure you've created the following directories and given yourself permissions on them: + +`/var/log/cassandra` + +`/var/lib/cassandra` + +2.) From the bdp root directory, run the following commands: + +`./gradlew jar` + +`bin/dse cassandra` + +3.) When the node stabilizes, open up `cqlsh` from the bdp root directory. + +`bin/cqlsh` + +### Create a Simple Data Model + +1.) Run the following DDL statements to create a table and two indexes: + +`CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy' , 'Cassandra': '1'};` + +`USE test;` + +`CREATE TABLE person (id int, name text, age int, PRIMARY KEY (id));` + +`CREATE CUSTOM INDEX ON person (name) USING 'StorageAttachedIndex' WITH OPTIONS = {'case_sensitive': false};` + +`CREATE CUSTOM INDEX ON person (age) USING 'StorageAttachedIndex';` + +2.) Add some data. + +`INSERT INTO person (id, name, age) VALUES (1, 'John', 21);` + +`INSERT INTO person (id, name, age) VALUES (2, 'john', 50);` + +`INSERT INTO person (id, name, age) VALUES (3, 'Boris', 43);` + +`INSERT INTO person (id, name, age) VALUES (4, 'Caleb', 34);` + +### Make Some Queries + +1.) Query for everyone named "John", ignoring case. + +`SELECT * FROM person WHERE name = 'John';` + +``` + id | age | name +----+-----+------ + 1 | 21 | John + 2 | 50 | john +``` + +2.) Query for everyone between the ages of 18 and 25. + +`SELECT * FROM person WHERE age >= 18 AND age <= 35;` + +``` + id | age | name +----+-----+------- + 1 | 21 | John + 4 | 34 | Caleb +``` + +## Contributors + +- Marc Selwan +- Caleb Rackliffe +- Zhao Yang +- Jason Rutherglen +- Maciej Zasada +- Andrew de la Peña +- Mike Adamson +- Zahir Patni +- Tomek Lasica +- Berenguer Blasi +- Rocco Varela diff --git a/src/java/org/apache/cassandra/index/sai/SSTableContext.java b/src/java/org/apache/cassandra/index/sai/SSTableContext.java new file mode 100644 index 000000000000..5543b3edd735 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/SSTableContext.java @@ -0,0 +1,290 @@ +/* + * 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.index.sai; + +import java.io.IOException; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import com.google.common.base.Objects; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.BlockPackedReader; +import org.apache.cassandra.index.sai.disk.v1.MetadataSource; +import org.apache.cassandra.index.sai.disk.v1.MonotonicBlockPackedReader; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.utils.Throwables; +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.index.sai.disk.OnDiskKeyProducer.NO_OFFSET; + +/** + * SSTableContext is created for individual sstable shared across indexes to track per-sstable index files. + * + * SSTableContext itself will be released when receiving sstable removed notification, but its shared copies in individual + * SSTableIndex will be released when in-flight read requests complete. + */ +public class SSTableContext extends SharedCloseableImpl +{ + public final SSTableReader sstable; + + private final IndexComponents groupComponents; + // mapping from sstable row id to token or offset + public final LongArray.Factory tokenReaderFactory, offsetReaderFactory; + public final KeyFetcher keyFetcher; + + private SSTableContext(SSTableReader sstable, + LongArray.Factory tokenReaderFactory, + LongArray.Factory offsetReaderFactory, + KeyFetcher keyFetcher, + Cleanup cleanup, + IndexComponents groupComponents) + { + super(cleanup); + this.sstable = sstable; + this.tokenReaderFactory = tokenReaderFactory; + this.offsetReaderFactory = offsetReaderFactory; + this.keyFetcher = keyFetcher; + this.groupComponents = groupComponents; + } + + private SSTableContext(SSTableContext copy) + { + super(copy); + this.sstable = copy.sstable; + this.tokenReaderFactory = copy.tokenReaderFactory; + this.offsetReaderFactory = copy.offsetReaderFactory; + this.groupComponents = copy.groupComponents; + this.keyFetcher = copy.keyFetcher; + } + + @SuppressWarnings("resource") + public static SSTableContext create(SSTableReader sstable) + { + IndexComponents groupComponents = IndexComponents.perSSTable(sstable); + + Ref sstableRef = null; + FileHandle token = null, offset = null; + LongArray.Factory tokenReaderFactory, offsetReaderFactory; + KeyFetcher keyFetcher; + try + { + MetadataSource source = MetadataSource.loadGroupMetadata(groupComponents); + + sstableRef = sstable.tryRef(); + + if (sstableRef == null) + { + throw new IllegalStateException("Couldn't acquire reference to the sstable: " + sstable); + } + + token = groupComponents.createFileHandle(IndexComponents.TOKEN_VALUES); + offset = groupComponents.createFileHandle(IndexComponents.OFFSETS_VALUES); + + tokenReaderFactory = new BlockPackedReader(token, IndexComponents.TOKEN_VALUES, groupComponents, source); + offsetReaderFactory = new MonotonicBlockPackedReader(offset, IndexComponents.OFFSETS_VALUES, groupComponents, source); + keyFetcher = new DecoratedKeyFetcher(sstable); + + Cleanup cleanup = new Cleanup(token, offset, sstableRef); + + return new SSTableContext(sstable, tokenReaderFactory, offsetReaderFactory, keyFetcher, cleanup, groupComponents); + } + catch (Throwable t) + { + if (sstableRef != null) + { + sstableRef.release(); + } + + throw Throwables.unchecked(Throwables.close(t, token, offset)); + } + } + + /** + * @return number of open files per {@link SSTableContext} instance + */ + public static int openFilesPerSSTable() + { + // token and offset + return 2; + } + + @Override + public SSTableContext sharedCopy() + { + return new SSTableContext(this); + } + + private static class Cleanup implements RefCounted.Tidy + { + private final FileHandle token, offset; + private final Ref sstableRef; + + private Cleanup(FileHandle token, FileHandle offset, Ref sstableRef) + { + this.token = token; + this.offset = offset; + this.sstableRef = sstableRef; + } + + @Override + public void tidy() + { + Throwable t = sstableRef.ensureReleased(null); + t = Throwables.close(t, token, offset); + + Throwables.maybeFail(t); + } + + @Override + public String name() + { + return null; + } + } + + /** + * @return descriptor of attached sstable + */ + public Descriptor descriptor() + { + return sstable.descriptor; + } + + public SSTableReader sstable() + { + return sstable; + } + + /** + * @return disk usage of per-sstable index files + */ + public long diskUsage() + { + return groupComponents.sizeOfPerSSTableComponents(); + } + + @Override + public String toString() + { + return "SSTableContext{" + + "sstable=" + sstable.descriptor + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SSTableContext that = (SSTableContext) o; + return Objects.equal(sstable.descriptor, that.sstable.descriptor); + } + + @Override + public int hashCode() + { + return Objects.hashCode(sstable.descriptor.hashCode()); + } + + public interface KeyFetcher + { + DecoratedKey apply(RandomAccessReader reader, long keyOffset); + + /** + * Create a shared RAR for all tokens in the same segment. + */ + RandomAccessReader createReader(); + } + + @VisibleForTesting + public static class DecoratedKeyFetcher implements KeyFetcher + { + private final SSTableReader sstable; + + DecoratedKeyFetcher(SSTableReader sstable) + { + this.sstable = sstable; + } + + @Override + public RandomAccessReader createReader() + { + return sstable.openKeyComponentReader(); + } + + @Override + public DecoratedKey apply(RandomAccessReader reader, long keyOffset) + { + assert reader != null : "RandomAccessReader null"; + + // If the returned offset is the sentinel value, we've seen this offset + // before or we've run out of valid keys due to ZCS: + if (keyOffset == NO_OFFSET) + return null; + + try + { + // can return null + return sstable.keyAt(reader, keyOffset); + } + catch (IOException e) + { + throw Throwables.cleaned(e); + } + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this).add("sstable", sstable).toString(); + } + + @Override + public int hashCode() + { + return sstable.descriptor.hashCode(); + } + + @Override + public boolean equals(Object other) + { + if (other == null) + { + return false; + } + if (other == this) + { + return true; + } + if (other.getClass() != getClass()) + { + return false; + } + DecoratedKeyFetcher rhs = (DecoratedKeyFetcher) other; + return sstable.descriptor.equals(rhs.sstable.descriptor); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/SSTableContextManager.java b/src/java/org/apache/cassandra/index/sai/SSTableContextManager.java new file mode 100644 index 000000000000..387403f90101 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/SSTableContextManager.java @@ -0,0 +1,143 @@ +/* + * 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.index.sai; + +import java.lang.invoke.MethodHandles; +import java.util.Collection; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import javax.annotation.concurrent.ThreadSafe; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Pair; + +/** + * Manage per-sstable {@link SSTableContext} for {@link StorageAttachedIndexGroup} + */ +@ThreadSafe +public class SSTableContextManager +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final ConcurrentHashMap sstableContexts = new ConcurrentHashMap<>(); + + /** + * Initialize {@link SSTableContext}s if they are not already initialized. + * + * @param removed SSTables being removed + * @param added SSTables being added + * @param validate if true, header and footer will be validated. + * + * @return a set of contexts for SSTables with valid per-SSTable components, and a set of + * SSTables with invalid or missing components + */ + @SuppressWarnings("resource") + public Pair, Set> update(Collection removed, Iterable added, boolean validate) + { + release(removed); + + Set contexts = new HashSet<>(); + Set invalid = new HashSet<>(); + + for (SSTableReader sstable : added) + { + if (sstable.isMarkedCompacted()) + { + continue; + } + + if (!IndexComponents.isGroupIndexComplete(sstable.descriptor)) + { + // Don't even try to validate or add the context if the completion marker is missing. + continue; + } + + try + { + // Only validate on restart or newly refreshed SSTable. Newly built files are unlikely to be corrupted. + if (validate && !sstableContexts.containsKey(sstable)) + { + IndexComponents.perSSTable(sstable).validatePerSSTableComponents(); + } + + // ConcurrentHashMap#computeIfAbsent guarantees atomicity, so {@link SSTableContext#create(SSTableReader)}} + // is called at most once per key. + contexts.add(sstableContexts.computeIfAbsent(sstable, SSTableContext::create)); + } + catch (Throwable t) + { + IndexComponents components = IndexComponents.perSSTable(sstable); + logger.warn(components.logMessage("Invalid per-SSTable component after sstable {} add.."), sstable.descriptor, t); + invalid.add(sstable); + SSTableContext failed = sstableContexts.remove(sstable); + if (failed != null) + { + failed.close(); + } + } + } + + return Pair.create(contexts, invalid); + } + + public void release(Collection toRelease) + { + toRelease.stream().map(sstableContexts::remove).filter(Objects::nonNull).forEach(SSTableContext::close); + } + + /** + * @return total number of per-sstable open files for live sstables + */ + int openFiles() + { + return size() * SSTableContext.openFilesPerSSTable(); + } + + /** + * @return total disk usage of all per-sstable index files + */ + long diskUsage() + { + return sstableContexts.values().stream().mapToLong(SSTableContext::diskUsage).sum(); + } + + Set sstables() + { + return sstableContexts.keySet(); + } + + @VisibleForTesting + public int size() + { + return sstableContexts.size(); + } + + @VisibleForTesting + public void clear() + { + sstableContexts.values().forEach(SSTableContext::close); + sstableContexts.clear(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/SSTableIndex.java b/src/java/org/apache/cassandra/index/sai/SSTableIndex.java new file mode 100644 index 000000000000..d2ec0addd19d --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/SSTableIndex.java @@ -0,0 +1,364 @@ +/* + * 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.index.sai; + +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import org.apache.commons.lang3.builder.HashCodeBuilder; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.index.sai.disk.Segment; +import org.apache.cassandra.index.sai.disk.SegmentMetadata; +import org.apache.cassandra.index.sai.disk.format.Version; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.io.IndexComponents.IndexComponent; +import org.apache.cassandra.index.sai.disk.v1.MetadataSource; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.RangeConcatIterator; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Throwables; + + +/** + * SSTableIndex is created for each column index on individual sstable to track per-column indexer. + */ +public class SSTableIndex +{ + // sort sstable index by first key then last key + public static final Comparator COMPARATOR = Comparator.comparing((SSTableIndex s) -> s.getSSTable().first) + .thenComparing(s -> s.getSSTable().last) + .thenComparing(s -> s.getSSTable().descriptor.generation); + + private final Version version; + private final SSTableContext sstableContext; + private final ColumnContext columnContext; + private final SSTableReader sstable; + private final IndexComponents components; + + private final ImmutableList segments; + private PerIndexFiles indexFiles; + + private final List metadatas; + private final DecoratedKey minKey, maxKey; // in token order + private final ByteBuffer minTerm, maxTerm; + private final long minSSTableRowId, maxSSTableRowId; + private final long numRows; + + private final AtomicInteger references = new AtomicInteger(1); + private final AtomicBoolean obsolete = new AtomicBoolean(false); + + public SSTableIndex(SSTableContext sstableContext, ColumnContext columnContext, IndexComponents components) + { + this.sstableContext = sstableContext.sharedCopy(); + this.columnContext = columnContext; + this.sstable = sstableContext.sstable; + this.components = components; + + final AbstractType validator = columnContext.getValidator(); + assert validator != null; + + try + { + this.indexFiles = new PerIndexFiles(components, columnContext.isLiteral()); + + ImmutableList.Builder segmentsBuilder = ImmutableList.builder(); + + final MetadataSource source = MetadataSource.loadColumnMetadata(components); + version = source.getVersion(); + metadatas = SegmentMetadata.load(source, null); + + for (SegmentMetadata metadata : metadatas) + { + segmentsBuilder.add(new Segment(columnContext, sstableContext, indexFiles, metadata)); + } + + segments = segmentsBuilder.build(); + assert !segments.isEmpty(); + + this.minKey = metadatas.get(0).minKey; + this.maxKey = metadatas.get(metadatas.size() - 1).maxKey; + + this.minTerm = metadatas.stream().map(m -> m.minTerm).min(TypeUtil.comparator(validator)).orElse(null); + this.maxTerm = metadatas.stream().map(m -> m.maxTerm).max(TypeUtil.comparator(validator)).orElse(null); + + this.numRows = metadatas.stream().mapToLong(m -> m.numRows).sum(); + + this.minSSTableRowId = metadatas.get(0).minSSTableRowId; + this.maxSSTableRowId = metadatas.get(metadatas.size() - 1).maxSSTableRowId; + } + catch (Throwable t) + { + FileUtils.closeQuietly(indexFiles); + FileUtils.closeQuietly(sstableContext); + throw Throwables.unchecked(t); + } + } + + public ColumnContext getColumnContext() + { + return columnContext; + } + + public SSTableContext getSSTableContext() + { + return sstableContext; + } + + public long indexFileCacheSize() + { + return segments.stream().mapToLong(Segment::indexFileCacheSize).sum(); + } + + /** + * @return number of indexed rows, note that rows may have been updated or removed in sstable. + */ + public long getRowCount() + { + return numRows; + } + + /** + * @return total size of per-column index components, in bytes + */ + public long sizeOfPerColumnComponents() + { + return components.sizeOfPerColumnComponents(); + } + + /** + * @return the smallest possible sstable row id in this index. + */ + public long minSSTableRowId() + { + return minSSTableRowId; + } + + /** + * @return the largest possible sstable row id in this index. + */ + public long maxSSTableRowId() + { + return maxSSTableRowId; + } + + public ByteBuffer minTerm() + { + return minTerm; + } + + public ByteBuffer maxTerm() + { + return maxTerm; + } + + public DecoratedKey minKey() + { + return minKey; + } + + public DecoratedKey maxKey() + { + return maxKey; + } + + public RangeIterator search(Expression expression, AbstractBounds keyRange, SSTableQueryContext context, boolean defer) + { + RangeConcatIterator.Builder builder = RangeConcatIterator.builder(); + + for (Segment segment : segments) + { + if (segment.intersects(keyRange)) + { + builder.add(segment.search(expression, context, defer)); + } + } + + return builder.build(); + } + + public int getSegmentSize() + { + return segments.size(); + } + + public List segments() + { + return metadatas; + } + + public Version getVersion() + { + return version; + } + + /** + * container to share per-index file handles(kdtree, terms data, posting lists) among segments. + */ + public static class PerIndexFiles implements Closeable + { + private final Map files = new HashMap<>(2); + private final IndexComponents components; + + public PerIndexFiles(IndexComponents components, boolean isStringIndex) + { + this(components, isStringIndex, false); + } + + public PerIndexFiles(IndexComponents components, boolean isStringIndex, boolean temporary) + { + this.components = components; + if (isStringIndex) + { + files.put(components.postingLists, components.createFileHandle(components.postingLists, temporary)); + files.put(components.termsData, components.createFileHandle(components.termsData, temporary)); + } + else + { + files.put(components.kdTree, components.createFileHandle(components.kdTree, temporary)); + files.put(components.kdTreePostingLists, components.createFileHandle(components.kdTreePostingLists, temporary)); + } + } + + public FileHandle kdtree() + { + return getFile(components.kdTree); + } + + public FileHandle postingLists() + { + return getFile(components.postingLists); + } + + public FileHandle termsData() + { + return getFile(components.termsData); + } + + public FileHandle kdtreePostingLists() + { + return getFile(components.kdTreePostingLists); + } + + private FileHandle getFile(IndexComponent type) + { + FileHandle file = files.get(type); + if (file == null) + throw new IllegalArgumentException(String.format("Component %s not found for SSTable %s", type.name, components.descriptor)); + + return file; + } + + public IndexComponents components() + { + return this.components; + } + + @Override + public void close() + { + FileUtils.closeQuietly(files.values()); + } + } + + public SSTableReader getSSTable() + { + return sstable; + } + + public boolean reference() + { + while (true) + { + int n = references.get(); + if (n <= 0) + return false; + if (references.compareAndSet(n, n + 1)) + { + return true; + } + } + } + + public boolean isReleased() + { + return references.get() <= 0; + } + + public void release() + { + int n = references.decrementAndGet(); + + if (n == 0) + { + FileUtils.closeQuietly(indexFiles); + FileUtils.closeQuietly(segments); + sstableContext.close(); + + /* + * When SSTable is removed, storage-attached index components will be automatically removed by LogTransaction. + * We only remove index components explicitly in case of index corruption or index rebuild. + */ + if (obsolete.get()) + { + components.deleteColumnIndex(); + } + } + } + + public void markObsolete() + { + obsolete.getAndSet(true); + release(); + } + + public boolean equals(Object o) + { + return o instanceof SSTableIndex && components.equals(((SSTableIndex) o).components); + } + + public int hashCode() + { + return new HashCodeBuilder().append(components.hashCode()).build(); + } + + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("column", columnContext.getColumnName()) + .add("sstable", sstable.descriptor) + .add("totalRows", sstable.getTotalRows()) + .toString(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/SSTableQueryContext.java b/src/java/org/apache/cassandra/index/sai/SSTableQueryContext.java new file mode 100644 index 000000000000..42177b323b40 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/SSTableQueryContext.java @@ -0,0 +1,63 @@ +/* + * 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.index.sai; + +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; + +/** + * Tracks SSTable-specific state relevant to the execution of a single query. + * + * Fields here are non-volatile, as they are accessed from a single thread. + */ +@NotThreadSafe +public class SSTableQueryContext +{ + public final QueryContext queryContext; + + // During intersection queries, multiple column indexes touch the same exact tokens as we skip + // between range iterators. Caching the values of these global SSTable-specific lookups allows us to avoid + // large chunks of duplicated work. + public long prevTokenValue = Long.MIN_VALUE; + public long prevSSTableRowId = -1; + + public long prevSkipToTokenValue = Long.MIN_VALUE; + public long prevSkipToSSTableRowId = -1; + + public SSTableQueryContext(QueryContext queryContext) + { + this.queryContext = queryContext; + } + + @VisibleForTesting + public static SSTableQueryContext forTest() + { + return new SSTableQueryContext(new QueryContext()); + } + + public void markTokenSkippingLookup() + { + queryContext.tokenSkippingLookups++; + } + + public void markTokenSkippingCacheHit() + { + queryContext.tokenSkippingCacheHits++; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/StorageAttachedIndex.java b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndex.java new file mode 100644 index 000000000000..729006600208 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndex.java @@ -0,0 +1,728 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Predicates; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Gauge; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQL3Type; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.cql3.statements.schema.IndexTarget; +import org.apache.cassandra.db.CassandraWriteContext; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.WriteContext; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.dht.ByteOrderedPartitioner; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.dht.OrderPreservingPartitioner; +import org.apache.cassandra.dht.RandomPartitioner; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.IndexRegistry; +import org.apache.cassandra.index.SecondaryIndexBuilder; +import org.apache.cassandra.index.SecondaryIndexManager; +import org.apache.cassandra.index.TargetParser; +import org.apache.cassandra.index.sai.analyzer.AbstractAnalyzer; +import org.apache.cassandra.index.sai.analyzer.NonTokenizingOptions; +import org.apache.cassandra.index.sai.disk.ColumnIndexWriter; +import org.apache.cassandra.index.sai.disk.IndexWriterConfig; +import org.apache.cassandra.index.sai.disk.MemtableIndexWriter; +import org.apache.cassandra.index.sai.disk.SSTableIndexWriter; +import org.apache.cassandra.index.sai.disk.SegmentBuilder; +import org.apache.cassandra.index.sai.disk.StorageAttachedIndexWriter; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.memory.RowMapping; +import org.apache.cassandra.index.sai.metrics.AbstractMetrics; +import org.apache.cassandra.index.sai.utils.NamedMemoryLimiter; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.index.sai.view.View; +import org.apache.cassandra.index.transactions.IndexTransaction; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.metrics.CassandraMetricsRegistry; +import org.apache.cassandra.metrics.DefaultNameFactory; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.schema.IndexMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.concurrent.OpOrder; + +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; + +public class StorageAttachedIndex implements Index +{ + private static final Logger logger = LoggerFactory.getLogger(StorageAttachedIndex.class); + + /** + * Global limit on heap consumed by all index segment building that occurs outside the context of Memtable flush. + * + * Note that to avoid flushing extremely small index segments, a segment is only flushed when + * both the global size of all building segments has breached the limit and the size of the + * segment in question reaches (segment_write_buffer_space_mb / # currently building column indexes). + * + * ex. If there is only one column index building, it can buffer up to segment_write_buffer_space_mb. + * + * ex. If there is one column index building per table across 8 compactors, each index will be + * eligible to flush once it reaches (segment_write_buffer_space_mb / 8) MBs. + */ + public static final long SEGMENT_BUILD_MEMORY_LIMIT = Long.getLong("cassandra.test.sai.segment_build_memory_limit", + 1024L * 1024L * (long) DatabaseDescriptor.getSAISegmentWriteBufferSpace()); + + public static final NamedMemoryLimiter SEGMENT_BUILD_MEMORY_LIMITER = + new NamedMemoryLimiter(SEGMENT_BUILD_MEMORY_LIMIT, "SSTable-attached Index Segment Builder"); + + static + { + CassandraMetricsRegistry.MetricName bufferSpaceUsed = DefaultNameFactory.createMetricName(AbstractMetrics.TYPE, "SegmentBufferSpaceUsedBytes", null); + CassandraMetricsRegistry.Metrics.register(bufferSpaceUsed, (Gauge) SEGMENT_BUILD_MEMORY_LIMITER::currentBytesUsed); + + CassandraMetricsRegistry.MetricName bufferSpaceLimit = DefaultNameFactory.createMetricName(AbstractMetrics.TYPE, "SegmentBufferSpaceLimitBytes", null); + CassandraMetricsRegistry.Metrics.register(bufferSpaceLimit, (Gauge) () -> SEGMENT_BUILD_MEMORY_LIMIT); + + // Note: The active builder count starts at 1 to avoid dividing by zero. + CassandraMetricsRegistry.MetricName buildsInProgress = DefaultNameFactory.createMetricName(AbstractMetrics.TYPE, "ColumnIndexBuildsInProgress", null); + CassandraMetricsRegistry.Metrics.register(buildsInProgress, (Gauge) () -> SegmentBuilder.ACTIVE_BUILDER_COUNT.get() - 1); + } + + private static class StorageAttachedIndexBuildingSupport implements IndexBuildingSupport + { + public SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs, + Set indexes, + Collection sstablesToRebuild, + boolean isFullRebuild) + { + NavigableMap> sstables = new TreeMap<>(Comparator.comparingInt(a -> a.descriptor.generation)); + StorageAttachedIndexGroup group = StorageAttachedIndexGroup.getIndexGroup(cfs); + + indexes.stream() + .filter((i) -> i instanceof StorageAttachedIndex) + .forEach((i) -> + { + StorageAttachedIndex sai = (StorageAttachedIndex) i; + ColumnContext context = ((StorageAttachedIndex) i).getContext(); + + // If this is not a full manual index rebuild we can skip SSTables that already have an + // attached index. Otherwise, we override any pre-existent index. + Collection ss = sstablesToRebuild; + if (!isFullRebuild) + { + ss = sstablesToRebuild.stream() + .filter(s -> !IndexComponents.isColumnIndexComplete(s.descriptor, context.getIndexName())) + .collect(Collectors.toList()); + } + + group.dropIndexSSTables(ss, sai); + + ss.forEach((sstable) -> + { + Set toBuild = sstables.get(sstable); + if (toBuild == null) sstables.put(sstable, (toBuild = new HashSet<>())); + toBuild.add(sai); + }); + }); + + return new StorageAttachedIndexBuilder(StorageAttachedIndexGroup.getIndexGroup(cfs), sstables, isFullRebuild, false); + } + } + + // Used to build indexes on newly added SSTables: + private static final StorageAttachedIndexBuildingSupport INDEX_BUILDER_SUPPORT = new StorageAttachedIndexBuildingSupport(); + + private static final Set VALID_OPTIONS = ImmutableSet.of(NonTokenizingOptions.CASE_SENSITIVE, + NonTokenizingOptions.NORMALIZE, + NonTokenizingOptions.ASCII, + IndexTarget.TARGET_OPTION_NAME, + IndexTarget.CUSTOM_INDEX_OPTION_NAME, + IndexWriterConfig.POSTING_LIST_LVL_MIN_LEAVES, + IndexWriterConfig.POSTING_LIST_LVL_SKIP_OPTION); + + public static final Set SUPPORTED_TYPES = ImmutableSet.of(CQL3Type.Native.ASCII, CQL3Type.Native.BIGINT, CQL3Type.Native.DATE, + CQL3Type.Native.DOUBLE, CQL3Type.Native.FLOAT, CQL3Type.Native.INT, + CQL3Type.Native.SMALLINT, CQL3Type.Native.TEXT, CQL3Type.Native.TIME, + CQL3Type.Native.TIMESTAMP, CQL3Type.Native.TIMEUUID, CQL3Type.Native.TINYINT, + CQL3Type.Native.UUID, CQL3Type.Native.VARCHAR, CQL3Type.Native.INET, + CQL3Type.Native.VARINT, CQL3Type.Native.DECIMAL, CQL3Type.Native.BOOLEAN); + + private static final Set> ILLEGAL_PARTITIONERS = + ImmutableSet.of(OrderPreservingPartitioner.class, LocalPartitioner.class, ByteOrderedPartitioner.class, RandomPartitioner.class); + + private final ColumnFamilyStore baseCfs; + private final IndexMetadata config; + private final ColumnContext context; + + // Tracks whether or not we've started the index build on initialization. + private volatile boolean initBuildStarted = false; + + // Tracks whether the index has been invalidated due to removal, a table drop, etc. + private volatile boolean valid = true; + + public StorageAttachedIndex(ColumnFamilyStore baseCfs, IndexMetadata config) + { + this.baseCfs = baseCfs; + this.config = config; + this.context = new ColumnContext(baseCfs.metadata(), config); + } + + /** + * Used via reflection in {@link IndexMetadata} + */ + @SuppressWarnings({ "unused" }) + public static Map validateOptions(Map options, TableMetadata metadata) + { + Map unknown = new HashMap<>(2); + + for (Map.Entry option : options.entrySet()) + { + if (!VALID_OPTIONS.contains(option.getKey())) + { + unknown.put(option.getKey(), option.getValue()); + } + } + + if (!unknown.isEmpty()) + { + return unknown; + } + + if (ILLEGAL_PARTITIONERS.contains(metadata.partitioner.getClass())) + { + throw new InvalidRequestException("Storage-attached index does not support the following IPartitioner implementations: " + ILLEGAL_PARTITIONERS); + } + + String targetColumn = options.get(IndexTarget.TARGET_OPTION_NAME); + + if (targetColumn == null) + { + throw new InvalidRequestException("Missing target column"); + } + + if (targetColumn.split(",").length > 1) + { + throw new InvalidRequestException("A storage-attached index cannot be created over multiple columns: " + targetColumn); + } + + Pair target = TargetParser.parse(metadata, targetColumn); + + if (target == null) + { + throw new InvalidRequestException("Failed to retrieve target column for: " + targetColumn); + } + + // In order to support different index target on non-frozen map, ie. KEYS, VALUE, ENTRIES, we need to put index + // name as part of index file name instead of column name. We only need to check that the target is different + // between indexes. This will only allow indexes in the same column with a different IndexTarget.Type. + // + // Note that: "metadata.indexes" already includes current index + if (metadata.indexes.stream().filter(index -> index.getIndexClassName().equals(StorageAttachedIndex.class.getName())) + .map(index -> TargetParser.parse(metadata, index.options.get(IndexTarget.TARGET_OPTION_NAME))) + .filter(Objects::nonNull).filter(t -> t.equals(target)).count() > 1) + { + throw new InvalidRequestException("Cannot create more than one storage-attached index on the same column: " + target.left); + } + + AbstractType type = TypeUtil.cellValueType(target); + + // If we are indexing map entries we need to validate the sub-types + if (TypeUtil.isComposite(type)) + { + for (AbstractType subType : type.subTypes()) + { + if (!SUPPORTED_TYPES.contains(subType.asCQL3Type()) && !TypeUtil.isFrozen(subType)) + throw new InvalidRequestException("Unsupported type: " + subType.asCQL3Type()); + } + } + else if (!SUPPORTED_TYPES.contains(type.asCQL3Type()) && !TypeUtil.isFrozen(type)) + { + throw new InvalidRequestException("Unsupported type: " + type.asCQL3Type()); + } + + AbstractAnalyzer.fromOptions(type, options); + IndexWriterConfig.fromOptions(null, type, options); + + return Collections.emptyMap(); + } + + @Override + public void register(IndexRegistry registry) + { + // index will be available for writes + registry.registerIndex(this, StorageAttachedIndexGroup.class, () -> new StorageAttachedIndexGroup(baseCfs)); + } + + @Override + public IndexMetadata getIndexMetadata() + { + return config; + } + + @Override + public Callable getInitializationTask() + { + // New storage-attached indexes will be available for queries after on disk index data are built. + // Memtable data will be indexed via flushing triggered by schema change + // We only want to validate the index files if we are starting up + return () -> startInitialBuild(baseCfs, StorageService.instance.isStarting()).get(); + } + + private Future startInitialBuild(ColumnFamilyStore baseCfs, boolean validate) + { + if (baseCfs.indexManager.isIndexQueryable(this)) + { + logger.debug(context.logMessage("Skipping validation and building in initialization task, as pre-join has already made the storage attached index queryable...")); + initBuildStarted = true; + return CompletableFuture.completedFuture(null); + } + + // stop in-progress compaction tasks to prevent compacted sstable not being index. + logger.debug(context.logMessage("Stopping active compactions to make sure all sstables are indexed after initial build.")); + CompactionManager.instance.interruptCompactionFor(Collections.singleton(baseCfs.metadata()), + OperationType.REWRITES_SSTABLES, + Predicates.alwaysTrue(), + true); + + // Force another flush to make sure on disk index is generated for memtable data before marking it queryable. + // In case of offline scrub, there is no live memtables. + if (!baseCfs.getTracker().getView().liveMemtables.isEmpty()) + { + baseCfs.forceBlockingFlush(ColumnFamilyStore.FlushReason.INDEX_BUILD_STARTED); + } + + // It is now safe to flush indexes directly from flushing Memtables. + initBuildStarted = true; + + StorageAttachedIndexGroup indexGroup = StorageAttachedIndexGroup.getIndexGroup(baseCfs); + List nonIndexed = findNonIndexedSSTables(baseCfs, indexGroup, validate, true); + + if (nonIndexed.isEmpty()) + { + return CompletableFuture.completedFuture(null); + } + + // split sorted sstables into groups with similar size and build each group in separate compaction thread + List> groups = groupBySize(nonIndexed, DatabaseDescriptor.getConcurrentCompactors()); + List> futures = new ArrayList<>(); + + for (List group : groups) + { + SortedMap> current = new TreeMap<>(Comparator.comparingLong(sstable -> sstable.descriptor.generation)); + group.forEach(sstable -> current.put(sstable, Collections.singleton(this))); + + futures.add(CompactionManager.instance.submitIndexBuild(new StorageAttachedIndexBuilder(indexGroup, current, false, true))); + } + + logger.info(context.logMessage("Submitting {} parallel initial index builds over {} total sstables..."), futures.size(), nonIndexed.size()); + return Futures.allAsList(futures); + } + + /** + * Splits SSTables into groups of similar overall size. + * + * @param toRebuild a list of SSTables to split (Note that this list will be sorted in place!) + * @param parallelism an upper bound on the number of groups + * + * @return a {@link List} of SSTable groups, each represented as a {@link List} of {@link SSTableReader} + */ + @VisibleForTesting + public static List> groupBySize(List toRebuild, int parallelism) + { + List> groups = new ArrayList<>(); + + toRebuild.sort(Comparator.comparingLong(SSTableReader::onDiskLength).reversed()); + Iterator sortedSSTables = toRebuild.iterator(); + double dataPerCompactor = toRebuild.stream().mapToLong(SSTableReader::onDiskLength).sum() * 1.0 / parallelism; + + while (sortedSSTables.hasNext()) + { + long sum = 0; + List current = new ArrayList<>(); + + while (sortedSSTables.hasNext() && sum < dataPerCompactor) + { + SSTableReader sstable = sortedSSTables.next(); + sum += sstable.onDiskLength(); + current.add(sstable); + } + + assert !current.isEmpty(); + groups.add(current); + } + + return groups; + } + + @Override + public Callable getMetadataReloadTask(IndexMetadata indexMetadata) + { + return null; + } + + @Override + public Callable getBlockingFlushTask() + { + return null; // storage-attached indexes are flushed alongside memtable + } + + @Override + public Callable getInvalidateTask() + { + return () -> + { + // mark index as invalid, in-progress SSTableIndexWriters will abort + valid = false; + + // in case of dropping table, SSTable indexes should already been removed by SSTableListChangedNotification. + Set toRemove = getComponents(); + for (SSTableIndex sstableIndex : context.getView().getIndexes()) + sstableIndex.getSSTable().unregisterComponents(toRemove, baseCfs.getTracker()); + + context.invalidate(); + return null; + }; + } + + @Override + public Callable getPreJoinTask(boolean hadBootstrap) + { + /* + * During bootstrap, streamed SSTable are already built for existing indexes via {@link StorageAttachedIndexBuildingSupport} + * from {@link org.apache.cassandra.streaming.StreamReceiveTask.OnCompletionRunnable}. + * + * For indexes created during bootstrapping, we don't have to block bootstrap for them. + */ + + return this::startPreJoinTask; + } + + private Future startPreJoinTask() + { + try + { + if (baseCfs.indexManager.isIndexQueryable(this)) + { + logger.debug(context.logMessage("Skipping validation in pre-join task, as the initialization task has already made the index queryable...")); + return null; + } + + StorageAttachedIndexGroup group = StorageAttachedIndexGroup.getIndexGroup(baseCfs); + Collection nonIndexed = findNonIndexedSSTables(baseCfs, group, true, true); + + if (nonIndexed.isEmpty()) + { + // If the index is complete, mark it queryable before the node starts accepting requests: + baseCfs.indexManager.makeIndexQueryable(this, Status.BUILD_SUCCEEDED); + } + } + catch (Throwable t) + { + logger.error(context.logMessage("Failed in pre-join task!"), t); + } + + return null; + } + + @Override + public Callable getTruncateTask(long truncatedAt) + { + /* + * index files will be removed as part of base sstable lifecycle in + * {@link LogTransaction#delete(java.io.File)} asynchronously. + */ + return null; + } + + @Override + public boolean shouldBuildBlocking() + { + return true; + } + + @Override + public Optional getBackingTable() + { + return Optional.empty(); + } + + @Override + public boolean dependsOn(ColumnMetadata column) + { + return context.getDefinition().compareTo(column) == 0; + } + + @Override + public boolean supportsExpression(ColumnMetadata column, Operator operator) + { + return dependsOn(column) && context.supports(operator); + } + + @Override + public AbstractType customExpressionValueType() + { + return null; + } + + @Override + public RowFilter getPostIndexQueryFilter(RowFilter filter) + { + // it should be executed from the SAI query plan, this is only used by the singleton index query plan + throw new UnsupportedOperationException(); + } + + @Override + public long getEstimatedResultRows() + { + throw new UnsupportedOperationException("Use StorageAttachedIndexQueryPlan#getEstimatedResultRows() instead."); + } + + @Override + public boolean isQueryable(Status status) + { + // consider unknown status as queryable, because gossip may not be up-to-date for newly joining nodes. + return status == Status.BUILD_SUCCEEDED || status == Status.UNKNOWN; + } + + @Override + public void validate(PartitionUpdate update) throws InvalidRequestException + {} + + /** + * This method is called by the startup tasks to find SSTables that don't have indexes. The method is + * synchronized so that the view is unchanged between validation and the selection of non-indexed SSTables. + * + * @return a list SSTables without attached indexes + */ + private synchronized List findNonIndexedSSTables(ColumnFamilyStore baseCfs, StorageAttachedIndexGroup group, boolean validate, boolean rename) + { + Set sstables = baseCfs.getLiveSSTables(); + + // Initialize the SSTable indexes w/ valid existing components... + assert group != null : "Missing index group on " + baseCfs.name; + group.onSSTableChanged(Collections.emptyList(), sstables, Collections.singleton(this), validate, rename); + + // ...then identify and rebuild the SSTable indexes that are missing. + List nonIndexed = new ArrayList<>(); + View view = context.getView(); + + for (SSTableReader sstable : sstables) + { + // An SSTable is considered not indexed if: + // 1. The current view does not contain the SSTable + // 2. The SSTable is not marked compacted + // 3. The column index does not have a completion marker + if (!view.containsSSTable(sstable) && !sstable.isMarkedCompacted() && + !IndexComponents.isColumnIndexComplete(sstable.descriptor, context.getIndexName())) + { + nonIndexed.add(sstable); + } + } + + return nonIndexed; + } + + private class UpdateIndexer extends IndexerAdapter + { + private final DecoratedKey key; + private final Memtable mt; + private final WriteContext writeContext; + + UpdateIndexer(DecoratedKey key, Memtable mt, WriteContext writeContext) + { + this.key = key; + this.mt = mt; + this.writeContext = writeContext; + } + + @Override + public void insertRow(Row row) + { + adjustMemtableSize(context.index(key, row, mt), CassandraWriteContext.fromContext(writeContext).getGroup()); + } + + @Override + public void updateRow(Row oldRow, Row newRow) + { + insertRow(newRow); + } + + void adjustMemtableSize(long additionalSpace, OpOrder.Group opGroup) + { + mt.markExtraOnHeapUsed(additionalSpace, opGroup); + } + } + + protected static abstract class IndexerAdapter implements Indexer + { + @Override + public void begin() { } + + @Override + public void finish() { } + + @Override + public void partitionDelete(DeletionTime dt) + { + } + + @Override + public void rangeTombstone(RangeTombstone rt) + { + } + + @Override + public void removeRow(Row row) + { + } + } + + @Override + public Searcher searcherFor(ReadCommand command) throws InvalidRequestException + { + // searchers should be created from the query plan, this is only used by the singleton index query plan + throw new UnsupportedOperationException(); + } + + @Override + public SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker) + { + throw new UnsupportedOperationException("Storage-attached index flush observers should never be created directly."); + } + + public ColumnIndexWriter newIndexWriter(Descriptor descriptor, + LifecycleNewTracker tracker, + RowMapping rowMapping, + CompressionParams compressionParams) + { + // If we're not flushing or we haven't yet started the initialization build, flush from SSTable contents. + if (tracker.opType() != OperationType.FLUSH || !initBuildStarted) + { + NamedMemoryLimiter limiter = SEGMENT_BUILD_MEMORY_LIMITER; + logger.info(context.logMessage("Starting a compaction index build. Global segment memory usage: {}"), prettyPrintMemory(limiter.currentBytesUsed())); + + return new SSTableIndexWriter(descriptor, context, limiter, () -> valid, compressionParams); + } + + return new MemtableIndexWriter(context.getPendingMemtableIndex(tracker), descriptor, context, rowMapping, compressionParams); + } + + @Override + public Set getComponents() + { + return new HashSet<>(IndexComponents.perColumnComponents(context.getIndexName(), context.isLiteral())); + } + + @Override + public Indexer indexerFor(DecoratedKey key, + RegularAndStaticColumns columns, + int nowInSec, + WriteContext writeContext, + IndexTransaction.Type transactionType, + Memtable memtable) + { + if (transactionType == IndexTransaction.Type.UPDATE) + { + return new UpdateIndexer(key, memtable, writeContext); + } + + // we are only interested in the data from Memtable + // everything else is going to be handled by SSTableWriter observers + return null; + } + + @Override + public IndexBuildingSupport getBuildTaskSupport() + { + return INDEX_BUILDER_SUPPORT; + } + + public ColumnContext getContext() + { + return context; + } + + @Override + public String toString() + { + return String.format("%s.%s.%s", baseCfs.keyspace.getName(), baseCfs.name, config == null ? "?" : config.name); + } + + /** + * Removes this index from the {@link SecondaryIndexManager}'s set of queryable indexes. + * + * This usually happens in response to an index writing failure from {@link StorageAttachedIndexWriter}. + */ + public void makeIndexNonQueryable() + { + baseCfs.indexManager.makeIndexNonQueryable(this, Status.BUILD_FAILED); + logger.warn(context.logMessage("Storage-attached index is no longer queryable. Please restart this node to repair it.")); + } + + void deleteIndexFiles(SSTableReader sstable) + { + IndexComponents.create(context.getIndexName(), sstable).deleteColumnIndex(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexBuilder.java b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexBuilder.java new file mode 100644 index 000000000000..91cbbdfcaa6c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexBuilder.java @@ -0,0 +1,370 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * + * 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.index.sai; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.stream.Collectors; + +import com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.compaction.CompactionInterruptedException; +import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.lifecycle.Tracker; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.index.SecondaryIndexBuilder; +import org.apache.cassandra.index.sai.disk.StorageAttachedIndexWriter; +import org.apache.cassandra.index.sai.disk.io.CryptoUtils; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableIdentityIterator; +import org.apache.cassandra.io.sstable.SSTableSimpleIterator; +import org.apache.cassandra.io.sstable.format.PartitionIndexIterator; +import org.apache.cassandra.io.sstable.format.RowIndexEntry; +import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.UUIDGen; +import org.apache.cassandra.utils.concurrent.Ref; + +import static org.apache.cassandra.db.compaction.TableOperation.StopTrigger.TRUNCATE; + +/** + * Multiple storage-attached indexes can start building concurrently. We need to make sure: + * 1. Per-SSTable index files are built only once, eg. {@link IndexComponents#PER_SSTABLE_COMPONENTS} + * a. Per-SSTable index files already built, do nothing + * b. Per-SSTable index files are currently building, we need to wait until it's built in order to consider index built. + * 2. Per-column index files are built for each column index..{@link IndexComponents#perColumnComponents(String, boolean)} + */ +public class StorageAttachedIndexBuilder extends SecondaryIndexBuilder +{ + protected static final Logger logger = LoggerFactory.getLogger(StorageAttachedIndexBuilder.class); + + // make sure only one builder can write to per sstable files when multiple storage-attached indexes are created simultaneously. + private static final Map inProgress = Maps.newConcurrentMap(); + + private final StorageAttachedIndexGroup group; + private final TableMetadata metadata; + private final Tracker tracker; + private final UUID compactionId = UUIDGen.getTimeUUID(); + private final boolean isFullRebuild; + private final boolean isInitialBuild; + + private final SortedMap> sstables; + + private long bytesProcessed = 0; + private final long totalSizeInBytes; + + StorageAttachedIndexBuilder(StorageAttachedIndexGroup group, SortedMap> sstables, boolean isFullRebuild, boolean isInitialBuild) + { + this.group = group; + this.metadata = group.metadata(); + this.sstables = sstables; + this.tracker = group.table().getTracker(); + this.isFullRebuild = isFullRebuild; + this.isInitialBuild = isInitialBuild; + this.totalSizeInBytes = sstables.keySet().stream().mapToLong(SSTableReader::uncompressedLength).sum(); + } + + @Override + public void build() + { + for (Map.Entry> e : sstables.entrySet()) + { + SSTableReader sstable = e.getKey(); + Set indexes = e.getValue(); + + Set existing = validateIndexes(indexes, sstable.descriptor); + if (existing.isEmpty()) + { + logger.debug(logMessage("{} dropped during index build"), indexes); + continue; + } + + if (indexSSTable(sstable, existing)) + { + return; + } + } + } + + private String logMessage(String message) { + return String.format("[%s.%s.*] %s", metadata.keyspace, metadata.name, message); + } + + /** + * @return true if index build should be stopped + */ + private boolean indexSSTable(SSTableReader sstable, Set indexes) + { + CountDownLatch perSSTableFileLock = null; + StorageAttachedIndexWriter indexWriter = null; + + Ref ref = sstable.tryRef(); + if (ref == null) + { + logger.warn(logMessage("Couldn't acquire reference to the SSTable {}. It may have been removed."), sstable.descriptor); + return false; + } + + try (RandomAccessReader dataFile = sstable.openDataReader(); + LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.INDEX_BUILD)) + { + perSSTableFileLock = shouldWriteTokenOffsetFiles(sstable); + boolean perColumnOnly = perSSTableFileLock == null; + // remove existing per column index files instead of overwriting + indexes.forEach(index -> index.deleteIndexFiles(sstable)); + + final CompressionParams compressionParams = CryptoUtils.getCompressionParams(sstable); + + indexWriter = new StorageAttachedIndexWriter(sstable.descriptor, indexes, txn, perColumnOnly, compressionParams); + + long previousKeyPosition = 0; + indexWriter.begin(); + + try (PartitionIndexIterator keys = sstable.allKeysIterator()) + { + while (!keys.isExhausted()) + { + if (isStopRequested()) + { + throw new CompactionInterruptedException(getProgress()); + } + + final DecoratedKey key = sstable.decorateKey(keys.key()); + final long keyPosition = keys.keyPosition(); + + indexWriter.startPartition(key, keyPosition); + + RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ); + dataFile.seek(indexEntry.position); + ByteBufferUtil.skipShortLength(dataFile); // key + + /* + * Not directly using {@link SSTableIdentityIterator#create(SSTableReader, FileDataInput, DecoratedKey)}, + * because we need to get position of partition level deletion and static row. + */ + long partitionDeletionPosition = dataFile.getFilePointer(); + DeletionTime partitionLevelDeletion = DeletionTime.serializer.deserialize(dataFile); + long staticRowPosition = dataFile.getFilePointer(); + + indexWriter.partitionLevelDeletion(partitionLevelDeletion, partitionDeletionPosition); + + DeserializationHelper helper = new DeserializationHelper(sstable.metadata(), sstable.descriptor.version.correspondingMessagingVersion(), DeserializationHelper.Flag.LOCAL); + + try (SSTableSimpleIterator iterator = SSTableSimpleIterator.create(sstable.metadata(), dataFile, sstable.header, helper, partitionLevelDeletion); + SSTableIdentityIterator partition = new SSTableIdentityIterator(sstable, key, partitionLevelDeletion, sstable.getFilename(), iterator)) + { + // if the row has statics attached, it has to be indexed separately + if (metadata.hasStaticColumns()) + indexWriter.nextUnfilteredCluster(partition.staticRow(), staticRowPosition); + + while (partition.hasNext()) + { + long unfilteredPosition = dataFile.getFilePointer(); + indexWriter.nextUnfilteredCluster(partition.next(), unfilteredPosition); + } + } + + keys.advance(); + long dataPosition = keys.isExhausted() ? sstable.uncompressedLength() : keys.dataPosition(); + bytesProcessed += dataPosition - previousKeyPosition; + previousKeyPosition = dataPosition; + } + + completeSSTable(indexWriter, sstable, indexes, perSSTableFileLock); + } + + return false; + } + catch (Throwable t) + { + if (indexWriter != null) + { + indexWriter.abort(t, true); + } + + if (t instanceof InterruptedException) + { + // TODO: Is there anything that makes more sense than just restoring the interrupt? + logger.warn(logMessage("Interrupted while building indexes {} on SSTable {}"), indexes, sstable.descriptor); + Thread.currentThread().interrupt(); + return true; + } + else if (t instanceof CompactionInterruptedException) + { + if (isInitialBuild && trigger() != TRUNCATE) + { + logger.error(logMessage("Stop requested while building initial indexes {} on SSTable {}."), indexes, sstable.descriptor); + throw Throwables.unchecked(t); + } + else + { + logger.info(logMessage("Stop requested while building indexes {} on SSTable {}."), indexes, sstable.descriptor); + return true; + } + } + else + { + logger.error(logMessage("Unable to build indexes {} on SSTable {}. Cause: {}."), indexes, sstable, t.getMessage()); + throw Throwables.unchecked(t); + } + } + finally + { + ref.release(); + // release current lock in case of error + if (perSSTableFileLock != null) + { + inProgress.remove(sstable); + perSSTableFileLock.countDown(); + } + } + } + + @Override + public OperationProgress getProgress() + { + return new OperationProgress(metadata, + OperationType.INDEX_BUILD, + bytesProcessed, + totalSizeInBytes, + compactionId, + sstables.keySet()); + } + + /** + * if the per sstable index files are already created, not need to write it again, unless found corrupted on rebuild + * if not created, try to acquire a lock, so only one builder will generate per sstable index files + */ + private CountDownLatch shouldWriteTokenOffsetFiles(SSTableReader sstable) + { + // if per-table files are incomplete or checksum failed during full rebuild. + if (!IndexComponents.isGroupIndexComplete(sstable.descriptor) || + (isFullRebuild && !IndexComponents.perSSTable(sstable).validatePerSSTableComponentsChecksum())) + { + CountDownLatch latch = new CountDownLatch(1); + if (inProgress.putIfAbsent(sstable, latch) == null) + { + // lock owner should cleanup existing per-SSTable files + group.deletePerSSTableFiles(Collections.singleton(sstable)); + return latch; + } + } + return null; + } + + private void completeSSTable(SSTableFlushObserver indexWriter, + SSTableReader sstable, + Set indexes, + CountDownLatch latch) throws InterruptedException + { + indexWriter.complete(); + + if (latch != null) + { + // current builder owns the lock + latch.countDown(); + } + else + { + /* + * When there is no lock, it means the per sstable index files are already created, just proceed to finish. + * When there is a lock held by another builder, wait for it to finish before finishing marking current index built. + */ + latch = inProgress.get(sstable); + if (latch != null) + latch.await(); + } + + Set existing = validateIndexes(indexes, sstable.descriptor); + if (existing.isEmpty()) + { + logger.debug(logMessage("{} dropped during index build"), indexes); + return; + } + + // register custom index components into existing sstables + sstable.registerComponents(group.getLiveComponents(sstable, existing), tracker); + Set incomplete = group.onSSTableChanged(Collections.emptyList(), Collections.singleton(sstable), existing, false, false); + + if (!incomplete.isEmpty()) + { + // If this occurs during an initial index build, there is only one index in play, and + // throwing here to terminate makes sense. (This allows the initialization task to fail + // correctly and be marked as failed by the SIM.) In other cases, such as rebuilding a + // set of indexes for a new added/streamed SSTables, we terminate pessimistically. In + // other words, we abort the SSTable index write across all column indexes and mark + // then non-queryable until a restart or other incremental rebuild occurs. + throw new RuntimeException(logMessage("Failed to update views on column indexes " + incomplete + " on indexes " + indexes + ".")); + } + } + + /** + * In case of full rebuild, stop the index build if any index is dropped. + * Otherwise, skip dropped indexes to avoid exception during repair/streaming. + */ + private Set validateIndexes(Set indexes, Descriptor descriptor) + { + Set existing = new HashSet<>(); + Set dropped = new HashSet<>(); + + for (StorageAttachedIndex index : indexes) + { + if (group.containsIndex(index)) + existing.add(index); + else + dropped.add(index); + } + + if (!dropped.isEmpty()) + { + String droppedIndexes = dropped.stream().map(sai -> sai.getContext().getIndexName()).collect(Collectors.toList()).toString(); + if (isFullRebuild) + throw new RuntimeException(logMessage(String.format("%s are dropped, will stop index build.", droppedIndexes))); + else + logger.debug(logMessage("Skip building dropped index {} on sstable {}"), droppedIndexes, descriptor.baseFilename()); + } + + return existing; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java new file mode 100644 index 000000000000..fe4dbd5f858b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java @@ -0,0 +1,434 @@ +/* + * 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.index.sai; + +import java.lang.invoke.MethodHandles; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.WriteContext; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.lifecycle.Tracker; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.sai.disk.StorageAttachedIndexWriter; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.metrics.IndexGroupMetrics; +import org.apache.cassandra.index.sai.metrics.TableQueryMetrics; +import org.apache.cassandra.index.sai.metrics.TableStateMetrics; +import org.apache.cassandra.index.sai.plan.StorageAttachedIndexQueryPlan; +import org.apache.cassandra.index.transactions.IndexTransaction; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.notifications.INotification; +import org.apache.cassandra.notifications.INotificationConsumer; +import org.apache.cassandra.notifications.MemtableDiscardedNotification; +import org.apache.cassandra.notifications.MemtableRenewedNotification; +import org.apache.cassandra.notifications.SSTableAddedNotification; +import org.apache.cassandra.notifications.SSTableListChangedNotification; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.Throwables; + +/** + * Orchestrates building of storage-attached indices, and manages lifecycle of resources shared between them. + */ +public class StorageAttachedIndexGroup implements Index.Group, INotificationConsumer +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final TableQueryMetrics queryMetrics; + private final TableStateMetrics stateMetrics; + private final IndexGroupMetrics groupMetrics; + + private final Set indices = new HashSet<>(); + private final ColumnFamilyStore baseCfs; + + private final SSTableContextManager contextManager; + + StorageAttachedIndexGroup(ColumnFamilyStore baseCfs) + { + this.baseCfs = baseCfs; + this.queryMetrics = new TableQueryMetrics(baseCfs.metadata()); + this.stateMetrics = new TableStateMetrics(baseCfs.metadata(), this); + this.groupMetrics = new IndexGroupMetrics(baseCfs.metadata(), this); + this.contextManager = new SSTableContextManager(); + + Tracker tracker = baseCfs.getTracker(); + tracker.subscribe(this); + } + + @Nullable + public static StorageAttachedIndexGroup getIndexGroup(ColumnFamilyStore cfs) + { + return (StorageAttachedIndexGroup)cfs.indexManager.getIndexGroup(StorageAttachedIndexGroup.class); + } + + @Override + public Set getIndexes() + { + return ImmutableSet.copyOf(indices); + } + + @Override + public void addIndex(Index index) + { + assert index instanceof StorageAttachedIndex; + indices.add((StorageAttachedIndex) index); + } + + @Override + public void removeIndex(Index index) + { + assert index instanceof StorageAttachedIndex; + boolean removed = indices.remove(index); + assert removed : "Cannot remove non-existing index " + index; + /* + * per index files are dropped via {@link StorageAttachedIndex#getInvalidateTask()} + */ + if (indices.isEmpty()) + { + Set toRemove = new HashSet<>(IndexComponents.PER_SSTABLE_COMPONENTS); + for (SSTableReader sstable : contextManager.sstables()) + sstable.unregisterComponents(toRemove, baseCfs.getTracker()); + + deletePerSSTableFiles(baseCfs.getLiveSSTables()); + baseCfs.getTracker().unsubscribe(this); + } + } + + @Override + public void invalidate() + { + // in case of dropping table, sstable contexts should already been removed by SSTableListChangedNotification. + queryMetrics.release(); + groupMetrics.release(); + stateMetrics.release(); + baseCfs.getTracker().unsubscribe(this); + } + + @Override + public boolean supportsMultipleContains() + { + return true; + } + + @Override + public boolean supportsDisjunction() + { + return true; + } + + @Override + public boolean containsIndex(Index index) + { + return index instanceof StorageAttachedIndex && indices.contains(index); + } + + @Override + public Index.Indexer indexerFor(Predicate indexSelector, + DecoratedKey key, + RegularAndStaticColumns columns, + int nowInSec, + WriteContext ctx, + IndexTransaction.Type transactionType, + Memtable memtable) + { + final Set indexers = + indices.stream().filter(indexSelector) + .map(i -> i.indexerFor(key, columns, nowInSec, ctx, transactionType, memtable)) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); + + return indexers.isEmpty() ? null : new StorageAttachedIndex.IndexerAdapter() + { + @Override + public void insertRow(Row row) + { + forEach(indexer -> indexer.insertRow(row)); + } + + @Override + public void updateRow(Row oldRow, Row newRow) + { + forEach(indexer -> indexer.updateRow(oldRow, newRow)); + } + + private void forEach(Consumer action) + { + indexers.forEach(action::accept); + } + }; + } + + @Override + public StorageAttachedIndexQueryPlan queryPlanFor(RowFilter rowFilter) + { + return StorageAttachedIndexQueryPlan.create(baseCfs, queryMetrics, indices, rowFilter); + } + + @Override + public SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker, TableMetadata tableMetadata) + { + try + { + return new StorageAttachedIndexWriter(descriptor, indices, tracker, tableMetadata.params.compression); + } + catch (Throwable t) + { + String message = "Unable to create storage-attached index writer on SSTable flush. All indexes from this table are going to be marked as non-queryable and will need to be rebuilt."; + logger.error(String.format("[%s.%s.*] %s", descriptor.ksname, descriptor.cfname, message), t); + indices.forEach(StorageAttachedIndex::makeIndexNonQueryable); + return null; + } + } + + @Override + public boolean handles(IndexTransaction.Type type) + { + // to skip CleanupGCTransaction and IndexGCTransaction + return type == IndexTransaction.Type.UPDATE; + } + + @Override + public Set getComponents() + { + return getComponents(indices); + } + + static Set getComponents(Collection indices) + { + Set components = new HashSet<>(IndexComponents.PER_SSTABLE_COMPONENTS); + indices.forEach(index -> components.addAll(index.getComponents())); + return components; + } + + // This differs from getComponents in that it only returns index components that exist on disk. + // It avoids errors being logged by the SSTable.readTOC method when we have an empty index. + @VisibleForTesting + public static Set getLiveComponents(SSTableReader sstable, Collection indices) + { + return getComponents(indices).stream() + .filter(component -> sstable.descriptor.fileFor(component).exists()) + .collect(Collectors.toSet()); + } + + @Override + public void handleNotification(INotification notification, Object sender) + { + // unfortunately, we can only check the type of notification via instanceof :( + if (notification instanceof SSTableAddedNotification) + { + SSTableAddedNotification notice = (SSTableAddedNotification) notification; + + // Avoid validation for index files just written following Memtable flush. ZCS streaming should + // validate index checksum. + boolean validate = !notice.memtable().isPresent(); + onSSTableChanged(Collections.emptySet(), notice.added, indices, validate, false); + } + else if (notification instanceof SSTableListChangedNotification) + { + SSTableListChangedNotification notice = (SSTableListChangedNotification) notification; + + // Avoid validation for index files just written during compaction. + onSSTableChanged(notice.removed, notice.added, indices, false, false); + } + else if (notification instanceof MemtableRenewedNotification) + { + indices.forEach(index -> index.getContext().renewMemtable(((MemtableRenewedNotification) notification).renewed)); + } + else if (notification instanceof MemtableDiscardedNotification) + { + indices.forEach(index -> index.getContext().discardMemtable(((MemtableDiscardedNotification) notification).memtable)); + } + } + + void deletePerSSTableFiles(Collection sstables) + { + contextManager.release(sstables); + sstables.forEach(sstableReader -> IndexComponents.deletePerSSTableIndexComponents(sstableReader.descriptor)); + } + + void dropIndexSSTables(Collection ss, StorageAttachedIndex index) + { + try + { + index.getContext().drop(ss); + } + catch (Throwable t) + { + // Mark the index non-queryable, as its view may be compromised. + index.makeIndexNonQueryable(); + + throw Throwables.unchecked(t); + } + } + + /** + * This method is synchronized to avoid concurrent initialization tasks validating same per-SSTable files. + * + * @return the set of column indexes that were marked as non-queryable as a result of their per-SSTable index + * files being corrupt or being unable to successfully update their views + */ + synchronized Set onSSTableChanged(Collection removed, Iterable added, + Set indexes, boolean validate, boolean rename) + { + Pair, Set> results = contextManager.update(removed, added, validate); + + if (!results.right.isEmpty()) + { + results.right.forEach(sstable -> { + IndexComponents.deletePerSSTableIndexComponents(sstable.descriptor); + // Column indexes are invalid if their SSTable-level components are corrupted so delete + // their associated index files and mark them non-queryable. + indices.forEach(index -> { + index.deleteIndexFiles(sstable); + index.makeIndexNonQueryable(); + }); + }); + return indices; + } + + Set incomplete = new HashSet<>(); + + for (StorageAttachedIndex index : indexes) + { + Set invalid = index.getContext().onSSTableChanged(removed, results.left, validate, rename); + + if (!invalid.isEmpty()) + { + // Delete the index files and mark the index non-queryable, as its view may be compromised, + // and incomplete, for our callers: + invalid.forEach(context -> index.deleteIndexFiles(context.sstable())); + index.makeIndexNonQueryable(); + incomplete.add(index); + } + } + return incomplete; + } + + /** + * open index files by checking number of {@link SSTableContext} and {@link SSTableIndex}, + * so transient open files during validation and files that are still open for in-flight requests will not be tracked. + * + * @return total number of open files for all {@link StorageAttachedIndex}es. + */ + public int openIndexFiles() + { + return contextManager.openFiles() + indices.stream().mapToInt(index -> index.getContext().openPerIndexFiles()).sum(); + } + + /** + * @return total disk usage of all per-sstable index files + */ + public long diskUsage() + { + return contextManager.diskUsage(); + } + + /** + * @return count of indexes building + */ + public int totalIndexBuildsInProgress() + { + return (int) indices.stream().filter(i -> baseCfs.indexManager.isIndexBuilding(i.getIndexMetadata().name)).count(); + } + + /** + * @return count of queryable indexes + */ + public int totalQueryableIndexCount() + { + return (int) indices.stream().filter(i -> baseCfs.indexManager.isIndexQueryable(i)).count(); + } + + /** + * @return count of indexes + */ + public int totalIndexCount() + { + return indices.size(); + } + + /** + * @return total disk usage of all per-sstable index files and per-column index files + */ + public long totalDiskUsage() + { + return diskUsage() + indices.stream().flatMap(i -> i.getContext().getView().getIndexes().stream()) + .mapToLong(SSTableIndex::sizeOfPerColumnComponents).sum(); + } + + public TableMetadata metadata() + { + return baseCfs.metadata(); + } + + public ColumnFamilyStore table() + { + return baseCfs; + } + + @VisibleForTesting + public SSTableContextManager sstableContextManager() + { + return contextManager; + } + + /** + * simulate index loading on restart with index file validation validation + */ + @VisibleForTesting + public void unsafeReload() + { + contextManager.clear(); + onSSTableChanged(baseCfs.getLiveSSTables(), Collections.emptySet(), indices, false, false); + onSSTableChanged(Collections.emptySet(), baseCfs.getLiveSSTables(), indices, true, true); + } + + /** + * Simulate the index going through a restart of node + */ + @VisibleForTesting + public void reset() + { + contextManager.clear(); + indices.forEach(index -> index.makeIndexNonQueryable()); + onSSTableChanged(baseCfs.getLiveSSTables(), Collections.emptySet(), indices, false, false); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/Token.java b/src/java/org/apache/cassandra/index/sai/Token.java new file mode 100644 index 000000000000..b73e61991778 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/Token.java @@ -0,0 +1,153 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.memory.InMemoryToken; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.utils.MergeIterator; + +/** + * A container that exposes an iterator of {@link DecoratedKey} from disk. It exists primarily + * because multiple partition keys may hash to the same token. + */ +@NotThreadSafe +public abstract class Token implements Comparable +{ + protected long token; + + public Token(long token) + { + this.token = token; + } + + /** + * Using Long instead long, because {@link RangeIterator} is based on Long and uses null to represent non-existing min/max. + */ + public Long get() + { + return token; + } + + @VisibleForTesting + public long getLong() + { + return token; + } + + public abstract Iterator keys(); + + @Override + public int compareTo(Token o) + { + return Long.compare(token, o.token); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this).add("token", token).toString(); + } + + /** + * This interface makes it possible for us to inject custom tokens and mergers in our tests. + */ + public interface TokenMerger + { + void add(Token other); + + Token merge(); + + default void reset() {} + } + + @NotThreadSafe + public static class ReusableTokenMerger implements TokenMerger + { + private final MergeIterator.Reducer reducer = MergeIterator.getIdentity(); + private final List> keyIterators; + private final List tokens; + + private Token firstToken; + + public ReusableTokenMerger(int capacity) + { + keyIterators = new ArrayList<>(capacity); + tokens = new ArrayList<>(capacity); + } + + @Override + public void add(Token token) + { + if (token == null) return; + + if (tokens.isEmpty()) + { + firstToken = token; + } + else + { + assert firstToken.token == token.token : "Adding keys with a different token!"; + } + + tokens.add(token); + } + + @Override + public Token merge() + { + assert firstToken != null : "No tokens have been added to this merger!"; + + if (tokens.size() == 1) return firstToken; + + // We don't materialize keys until we know a merge is necessary. + for (Token token : tokens) + { + keyIterators.add(token.keys()); + } + + return new InMemoryToken(firstToken.token, MergeIterator.get(keyIterators, DecoratedKey.comparator, reducer)); + } + + /** + * Clears the state of the merger, preparing it to consume a new group of {@link Token}s. + */ + public void reset() + { + keyIterators.clear(); + tokens.clear(); + firstToken = null; + reducer.onKeyChange(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/AbstractAnalyzer.java b/src/java/org/apache/cassandra/index/sai/analyzer/AbstractAnalyzer.java new file mode 100644 index 000000000000..e415bae48cc2 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/AbstractAnalyzer.java @@ -0,0 +1,116 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.analyzer; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; + +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.sai.utils.TypeUtil; + +public abstract class AbstractAnalyzer implements Iterator +{ + public static final Set> ANALYZABLE_TYPES = ImmutableSet.of(UTF8Type.instance, AsciiType.instance); + + protected ByteBuffer next = null; + String nextLiteral = null; + + /** + * @return true if index value is transformed, eg. normalized or lower-cased or tokenized. + */ + public abstract boolean transformValue(); + + /** + * Note: This method does not advance, as we rely on {@link #hasNext()} to buffer the next value. + * + * @return the raw value currently buffered by this iterator + */ + public ByteBuffer next() + { + if (next == null) + throw new NoSuchElementException(); + return next; + } + + /** + * Note: This method does not advance, as we rely on {@link #hasNext()} to buffer the next value. + * + * @return the string value currently buffered by this iterator + */ + public String nextLiteral(AbstractType validator) + { + if (nextLiteral != null) + { + return nextLiteral; + } + + assert next != null; + return TypeUtil.getString(next, validator); + } + + public void remove() + { + throw new UnsupportedOperationException(); + } + + protected abstract void resetInternal(ByteBuffer input); + + public void reset(ByteBuffer input) + { + this.next = null; + this.nextLiteral = null; + + resetInternal(input); + } + + public static AbstractAnalyzer fromOptions(AbstractType type, Map options) + { + if (hasNonTokenizingOptions(options)) + { + if (TypeUtil.isIn(type, ANALYZABLE_TYPES)) + { + return new NonTokenizingAnalyzer(type, options); + } + else + { + throw new InvalidRequestException("CQL type " + type.asCQL3Type() + " cannot be analyzed."); + } + } + return new NoOpAnalyzer(); + } + + private static boolean hasNonTokenizingOptions(Map options) + { + return options.get(NonTokenizingOptions.ASCII) != null || options.containsKey(NonTokenizingOptions.CASE_SENSITIVE) || options.containsKey(NonTokenizingOptions.NORMALIZE); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/NoOpAnalyzer.java b/src/java/org/apache/cassandra/index/sai/analyzer/NoOpAnalyzer.java new file mode 100644 index 000000000000..d8ae78ae6297 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/NoOpAnalyzer.java @@ -0,0 +1,68 @@ +/* + * 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.index.sai.analyzer; + +import java.nio.ByteBuffer; + +import com.google.common.base.MoreObjects; + +/** + * Default noOp tokenizer. The iterator will iterate only once + * returning the unmodified input + */ +public class NoOpAnalyzer extends AbstractAnalyzer +{ + private ByteBuffer input; + private boolean hasNext = false; + + @SuppressWarnings("unused") + NoOpAnalyzer() {} + + @Override + public boolean hasNext() + { + if (hasNext) + { + this.next = input; + this.hasNext = false; + return true; + } + this.next = null; + return false; + } + + @Override + protected void resetInternal(ByteBuffer input) + { + this.input = input; + this.hasNext = true; + } + + @Override + public boolean transformValue() + { + return false; + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this).toString(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/NonTokenizingAnalyzer.java b/src/java/org/apache/cassandra/index/sai/analyzer/NonTokenizingAnalyzer.java new file mode 100644 index 000000000000..b493ffab3f57 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/NonTokenizingAnalyzer.java @@ -0,0 +1,160 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.analyzer; + +import java.nio.ByteBuffer; +import java.util.Map; + +import com.google.common.base.MoreObjects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.analyzer.filter.BasicResultFilters; +import org.apache.cassandra.index.sai.analyzer.filter.FilterPipelineBuilder; +import org.apache.cassandra.index.sai.analyzer.filter.FilterPipelineExecutor; +import org.apache.cassandra.index.sai.analyzer.filter.FilterPipelineTask; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * Analyzer that does *not* tokenize the input. Optionally will + * apply filters for the input output as defined in analyzers options + */ +public class NonTokenizingAnalyzer extends AbstractAnalyzer +{ + private static final Logger logger = LoggerFactory.getLogger(NonTokenizingAnalyzer.class); + + private AbstractType type; + private NonTokenizingOptions options; + private FilterPipelineTask filterPipeline; + + private ByteBuffer input; + private boolean hasNext = false; + + NonTokenizingAnalyzer(AbstractType type, Map options) + { + this(type, NonTokenizingOptions.fromMap(options)); + } + + NonTokenizingAnalyzer(AbstractType type, NonTokenizingOptions tokenizerOptions) + { + this.type = type; + this.options = tokenizerOptions; + this.filterPipeline = getFilterPipeline(); + } + + @Override + public boolean hasNext() + { + // check that we know how to handle the input, otherwise bail + if (!TypeUtil.isIn(type, ANALYZABLE_TYPES)) return false; + + if (hasNext) + { + try + { + String input = type.getString(this.input); + + if (input == null) + { + throw new MarshalException(String.format("'null' deserialized value for %s with %s", + ByteBufferUtil.bytesToHex(this.input), type)); + } + + String result = FilterPipelineExecutor.execute(filterPipeline, input); + + if (result == null) + { + nextLiteral = null; + next = null; + return false; + } + + nextLiteral = result; + next = type.fromString(result); + + return true; + } + catch (MarshalException e) + { + logger.error("Failed to deserialize value with " + type, e); + return false; + } + finally + { + hasNext = false; + } + } + + return false; + } + + @Override + public boolean transformValue() + { + return !options.isCaseSensitive() || options.isNormalized() || options.isAscii(); + } + + @Override + protected void resetInternal(ByteBuffer input) + { + this.input = input; + this.hasNext = true; + } + + private FilterPipelineTask getFilterPipeline() + { + FilterPipelineBuilder builder = new FilterPipelineBuilder(new BasicResultFilters.NoOperation()); + + if (!options.isCaseSensitive()) + { + builder = builder.add("to_lower", new BasicResultFilters.LowerCase()); + } + + if (options.isNormalized()) + { + builder = builder.add("normalize", new BasicResultFilters.Normalize()); + } + + if (options.isAscii()) + { + builder = builder.add("ascii", new BasicResultFilters.Ascii()); + } + + return builder.build(); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("caseSensitive", options.isCaseSensitive()) + .add("normalized", options.isNormalized()) + .add("ascii", options.isAscii()) + .toString(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/NonTokenizingOptions.java b/src/java/org/apache/cassandra/index/sai/analyzer/NonTokenizingOptions.java new file mode 100644 index 000000000000..b13ce12b687a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/NonTokenizingOptions.java @@ -0,0 +1,162 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.analyzer; + +import java.util.HashMap; +import java.util.Map; + +import com.google.common.base.Strings; + +import org.apache.cassandra.exceptions.InvalidRequestException; + +public class NonTokenizingOptions +{ + public static final String NORMALIZE = "normalize"; + public static final String CASE_SENSITIVE = "case_sensitive"; + public static final String ASCII = "ascii"; + + private boolean caseSensitive; + private boolean normalized; + private boolean ascii; + + boolean isCaseSensitive() + { + return caseSensitive; + } + + void setCaseSensitive(boolean caseSensitive) + { + this.caseSensitive = caseSensitive; + } + + boolean isNormalized() + { + return this.normalized; + } + + void setAscii(boolean ascii) + { + this.ascii = ascii; + } + + boolean isAscii() + { + return this.ascii; + } + + void setNormalized(boolean normalized) + { + this.normalized = normalized; + } + + public static class OptionsBuilder + { + private boolean caseSensitive = true; + private boolean normalized = false; + private boolean ascii = false; + + OptionsBuilder() {} + + OptionsBuilder caseSensitive(boolean caseSensitive) + { + this.caseSensitive = caseSensitive; + return this; + } + + OptionsBuilder ascii(boolean ascii) + { + this.ascii = ascii; + return this; + } + + OptionsBuilder normalized(boolean normalized) + { + this.normalized = normalized; + return this; + } + + public NonTokenizingOptions build() + { + NonTokenizingOptions options = new NonTokenizingOptions(); + options.setCaseSensitive(caseSensitive); + options.setNormalized(normalized); + options.setAscii(ascii); + return options; + } + } + + public static NonTokenizingOptions getDefaultOptions() + { + return fromMap(new HashMap(1)); + } + + public static NonTokenizingOptions fromMap(Map options) + { + OptionsBuilder builder = new OptionsBuilder(); + + for (Map.Entry entry : options.entrySet()) + { + switch (entry.getKey()) + { + case CASE_SENSITIVE: + { + boolean boolValue = validateBoolean(entry.getValue(), CASE_SENSITIVE); + builder = builder.caseSensitive(boolValue); + break; + } + + case NORMALIZE: + { + boolean boolValue = validateBoolean(entry.getValue(), NORMALIZE); + builder = builder.normalized(boolValue); + break; + } + + case ASCII: + { + boolean boolValue = validateBoolean(entry.getValue(), ASCII); + builder = builder.ascii(boolValue); + break; + } + } + } + return builder.build(); + } + + private static boolean validateBoolean(String value, String option) + { + if (Strings.isNullOrEmpty(value)) + { + throw new InvalidRequestException("Empty value for boolean option '" + option + "'"); + } + + if (!value.equalsIgnoreCase(Boolean.TRUE.toString()) && !value.equalsIgnoreCase(Boolean.FALSE.toString())) + { + throw new InvalidRequestException("Illegal value for boolean option '" + option + "': " + value); + } + + return Boolean.parseBoolean(value); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/filter/BasicResultFilters.java b/src/java/org/apache/cassandra/index/sai/analyzer/filter/BasicResultFilters.java new file mode 100644 index 000000000000..b5ad225f65fe --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/filter/BasicResultFilters.java @@ -0,0 +1,2007 @@ +/* + * 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.index.sai.analyzer.filter; + +import java.text.Normalizer; +import java.util.Locale; + +/** + * Basic/General Token Filters + */ +public class BasicResultFilters +{ + private static final Locale DEFAULT_LOCALE = Locale.getDefault(); + + public static class LowerCase extends FilterPipelineTask + { + private final Locale locale; + + public LowerCase() + { + this.locale = DEFAULT_LOCALE; + } + + public String process(String input) + { + return input.toLowerCase(locale); + } + } + + public static class Normalize extends FilterPipelineTask + { + public Normalize() { } + + public String process(String input) + { + if (input == null) return null; + return Normalizer.isNormalized(input, Normalizer.Form.NFC) ? input : Normalizer.normalize(input, Normalizer.Form.NFC); + } + } + + public static class Ascii extends FilterPipelineTask + { + public Ascii() { } + + public String process(String input) + { + if (input == null) return null; + char[] inputChars = input.toCharArray(); + // The output can (potentially) be 4 times the size of the input + char[] outputChars = new char[inputChars.length * 4]; + int outputSize = foldToASCII(inputChars, 0, outputChars, 0, inputChars.length); + return new String(outputChars, 0, outputSize); + } + } + + public static class NoOperation extends FilterPipelineTask + { + public String process(String input) + { + return input; + } + } + + // copied from lucene org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter + public static final int foldToASCII(char input[], int inputPos, char output[], int outputPos, int length) + { + final int end = inputPos + length; + for (int pos = inputPos; pos < end ; ++pos) { + final char c = input[pos]; + + // Quick test: if it's not in range then just keep current character + if (c < '\u0080') { + output[outputPos++] = c; + } else { + switch (c) { + case '\u00C0': // À [LATIN CAPITAL LETTER A WITH GRAVE] + case '\u00C1': // Á [LATIN CAPITAL LETTER A WITH ACUTE] + case '\u00C2': //  [LATIN CAPITAL LETTER A WITH CIRCUMFLEX] + case '\u00C3': // à [LATIN CAPITAL LETTER A WITH TILDE] + case '\u00C4': // Ä [LATIN CAPITAL LETTER A WITH DIAERESIS] + case '\u00C5': // Å [LATIN CAPITAL LETTER A WITH RING ABOVE] + case '\u0100': // Ā [LATIN CAPITAL LETTER A WITH MACRON] + case '\u0102': // Ă [LATIN CAPITAL LETTER A WITH BREVE] + case '\u0104': // Ą [LATIN CAPITAL LETTER A WITH OGONEK] + case '\u018F': // Ə http://en.wikipedia.org/wiki/Schwa [LATIN CAPITAL LETTER SCHWA] + case '\u01CD': // Ǎ [LATIN CAPITAL LETTER A WITH CARON] + case '\u01DE': // Ǟ [LATIN CAPITAL LETTER A WITH DIAERESIS AND MACRON] + case '\u01E0': // Ǡ [LATIN CAPITAL LETTER A WITH DOT ABOVE AND MACRON] + case '\u01FA': // Ǻ [LATIN CAPITAL LETTER A WITH RING ABOVE AND ACUTE] + case '\u0200': // Ȁ [LATIN CAPITAL LETTER A WITH DOUBLE GRAVE] + case '\u0202': // Ȃ [LATIN CAPITAL LETTER A WITH INVERTED BREVE] + case '\u0226': // Ȧ [LATIN CAPITAL LETTER A WITH DOT ABOVE] + case '\u023A': // Ⱥ [LATIN CAPITAL LETTER A WITH STROKE] + case '\u1D00': // ᴀ [LATIN LETTER SMALL CAPITAL A] + case '\u1E00': // Ḁ [LATIN CAPITAL LETTER A WITH RING BELOW] + case '\u1EA0': // Ạ [LATIN CAPITAL LETTER A WITH DOT BELOW] + case '\u1EA2': // Ả [LATIN CAPITAL LETTER A WITH HOOK ABOVE] + case '\u1EA4': // Ấ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND ACUTE] + case '\u1EA6': // Ầ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND GRAVE] + case '\u1EA8': // Ẩ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND HOOK ABOVE] + case '\u1EAA': // Ẫ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND TILDE] + case '\u1EAC': // Ậ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND DOT BELOW] + case '\u1EAE': // Ắ [LATIN CAPITAL LETTER A WITH BREVE AND ACUTE] + case '\u1EB0': // Ằ [LATIN CAPITAL LETTER A WITH BREVE AND GRAVE] + case '\u1EB2': // Ẳ [LATIN CAPITAL LETTER A WITH BREVE AND HOOK ABOVE] + case '\u1EB4': // Ẵ [LATIN CAPITAL LETTER A WITH BREVE AND TILDE] + case '\u1EB6': // Ặ [LATIN CAPITAL LETTER A WITH BREVE AND DOT BELOW] + case '\u24B6': // Ⓐ [CIRCLED LATIN CAPITAL LETTER A] + case '\uFF21': // A [FULLWIDTH LATIN CAPITAL LETTER A] + output[outputPos++] = 'A'; + break; + case '\u00E0': // à [LATIN SMALL LETTER A WITH GRAVE] + case '\u00E1': // á [LATIN SMALL LETTER A WITH ACUTE] + case '\u00E2': // â [LATIN SMALL LETTER A WITH CIRCUMFLEX] + case '\u00E3': // ã [LATIN SMALL LETTER A WITH TILDE] + case '\u00E4': // ä [LATIN SMALL LETTER A WITH DIAERESIS] + case '\u00E5': // å [LATIN SMALL LETTER A WITH RING ABOVE] + case '\u0101': // ā [LATIN SMALL LETTER A WITH MACRON] + case '\u0103': // ă [LATIN SMALL LETTER A WITH BREVE] + case '\u0105': // ą [LATIN SMALL LETTER A WITH OGONEK] + case '\u01CE': // ǎ [LATIN SMALL LETTER A WITH CARON] + case '\u01DF': // ǟ [LATIN SMALL LETTER A WITH DIAERESIS AND MACRON] + case '\u01E1': // ǡ [LATIN SMALL LETTER A WITH DOT ABOVE AND MACRON] + case '\u01FB': // ǻ [LATIN SMALL LETTER A WITH RING ABOVE AND ACUTE] + case '\u0201': // ȁ [LATIN SMALL LETTER A WITH DOUBLE GRAVE] + case '\u0203': // ȃ [LATIN SMALL LETTER A WITH INVERTED BREVE] + case '\u0227': // ȧ [LATIN SMALL LETTER A WITH DOT ABOVE] + case '\u0250': // ɐ [LATIN SMALL LETTER TURNED A] + case '\u0259': // ə [LATIN SMALL LETTER SCHWA] + case '\u025A': // ɚ [LATIN SMALL LETTER SCHWA WITH HOOK] + case '\u1D8F': // ᶏ [LATIN SMALL LETTER A WITH RETROFLEX HOOK] + case '\u1D95': // ᶕ [LATIN SMALL LETTER SCHWA WITH RETROFLEX HOOK] + case '\u1E01': // ạ [LATIN SMALL LETTER A WITH RING BELOW] + case '\u1E9A': // ả [LATIN SMALL LETTER A WITH RIGHT HALF RING] + case '\u1EA1': // ạ [LATIN SMALL LETTER A WITH DOT BELOW] + case '\u1EA3': // ả [LATIN SMALL LETTER A WITH HOOK ABOVE] + case '\u1EA5': // ấ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND ACUTE] + case '\u1EA7': // ầ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND GRAVE] + case '\u1EA9': // ẩ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND HOOK ABOVE] + case '\u1EAB': // ẫ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND TILDE] + case '\u1EAD': // ậ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND DOT BELOW] + case '\u1EAF': // ắ [LATIN SMALL LETTER A WITH BREVE AND ACUTE] + case '\u1EB1': // ằ [LATIN SMALL LETTER A WITH BREVE AND GRAVE] + case '\u1EB3': // ẳ [LATIN SMALL LETTER A WITH BREVE AND HOOK ABOVE] + case '\u1EB5': // ẵ [LATIN SMALL LETTER A WITH BREVE AND TILDE] + case '\u1EB7': // ặ [LATIN SMALL LETTER A WITH BREVE AND DOT BELOW] + case '\u2090': // ₐ [LATIN SUBSCRIPT SMALL LETTER A] + case '\u2094': // ₔ [LATIN SUBSCRIPT SMALL LETTER SCHWA] + case '\u24D0': // ⓐ [CIRCLED LATIN SMALL LETTER A] + case '\u2C65': // ⱥ [LATIN SMALL LETTER A WITH STROKE] + case '\u2C6F': // Ɐ [LATIN CAPITAL LETTER TURNED A] + case '\uFF41': // a [FULLWIDTH LATIN SMALL LETTER A] + output[outputPos++] = 'a'; + break; + case '\uA732': // Ꜳ [LATIN CAPITAL LETTER AA] + output[outputPos++] = 'A'; + output[outputPos++] = 'A'; + break; + case '\u00C6': // Æ [LATIN CAPITAL LETTER AE] + case '\u01E2': // Ǣ [LATIN CAPITAL LETTER AE WITH MACRON] + case '\u01FC': // Ǽ [LATIN CAPITAL LETTER AE WITH ACUTE] + case '\u1D01': // ᴁ [LATIN LETTER SMALL CAPITAL AE] + output[outputPos++] = 'A'; + output[outputPos++] = 'E'; + break; + case '\uA734': // Ꜵ [LATIN CAPITAL LETTER AO] + output[outputPos++] = 'A'; + output[outputPos++] = 'O'; + break; + case '\uA736': // Ꜷ [LATIN CAPITAL LETTER AU] + output[outputPos++] = 'A'; + output[outputPos++] = 'U'; + break; + case '\uA738': // Ꜹ [LATIN CAPITAL LETTER AV] + case '\uA73A': // Ꜻ [LATIN CAPITAL LETTER AV WITH HORIZONTAL BAR] + output[outputPos++] = 'A'; + output[outputPos++] = 'V'; + break; + case '\uA73C': // Ꜽ [LATIN CAPITAL LETTER AY] + output[outputPos++] = 'A'; + output[outputPos++] = 'Y'; + break; + case '\u249C': // ⒜ [PARENTHESIZED LATIN SMALL LETTER A] + output[outputPos++] = '('; + output[outputPos++] = 'a'; + output[outputPos++] = ')'; + break; + case '\uA733': // ꜳ [LATIN SMALL LETTER AA] + output[outputPos++] = 'a'; + output[outputPos++] = 'a'; + break; + case '\u00E6': // æ [LATIN SMALL LETTER AE] + case '\u01E3': // ǣ [LATIN SMALL LETTER AE WITH MACRON] + case '\u01FD': // ǽ [LATIN SMALL LETTER AE WITH ACUTE] + case '\u1D02': // ᴂ [LATIN SMALL LETTER TURNED AE] + output[outputPos++] = 'a'; + output[outputPos++] = 'e'; + break; + case '\uA735': // ꜵ [LATIN SMALL LETTER AO] + output[outputPos++] = 'a'; + output[outputPos++] = 'o'; + break; + case '\uA737': // ꜷ [LATIN SMALL LETTER AU] + output[outputPos++] = 'a'; + output[outputPos++] = 'u'; + break; + case '\uA739': // ꜹ [LATIN SMALL LETTER AV] + case '\uA73B': // ꜻ [LATIN SMALL LETTER AV WITH HORIZONTAL BAR] + output[outputPos++] = 'a'; + output[outputPos++] = 'v'; + break; + case '\uA73D': // ꜽ [LATIN SMALL LETTER AY] + output[outputPos++] = 'a'; + output[outputPos++] = 'y'; + break; + case '\u0181': // Ɓ [LATIN CAPITAL LETTER B WITH HOOK] + case '\u0182': // Ƃ [LATIN CAPITAL LETTER B WITH TOPBAR] + case '\u0243': // Ƀ [LATIN CAPITAL LETTER B WITH STROKE] + case '\u0299': // ʙ [LATIN LETTER SMALL CAPITAL B] + case '\u1D03': // ᴃ [LATIN LETTER SMALL CAPITAL BARRED B] + case '\u1E02': // Ḃ [LATIN CAPITAL LETTER B WITH DOT ABOVE] + case '\u1E04': // Ḅ [LATIN CAPITAL LETTER B WITH DOT BELOW] + case '\u1E06': // Ḇ [LATIN CAPITAL LETTER B WITH LINE BELOW] + case '\u24B7': // Ⓑ [CIRCLED LATIN CAPITAL LETTER B] + case '\uFF22': // B [FULLWIDTH LATIN CAPITAL LETTER B] + output[outputPos++] = 'B'; + break; + case '\u0180': // ƀ [LATIN SMALL LETTER B WITH STROKE] + case '\u0183': // ƃ [LATIN SMALL LETTER B WITH TOPBAR] + case '\u0253': // ɓ [LATIN SMALL LETTER B WITH HOOK] + case '\u1D6C': // ᵬ [LATIN SMALL LETTER B WITH MIDDLE TILDE] + case '\u1D80': // ᶀ [LATIN SMALL LETTER B WITH PALATAL HOOK] + case '\u1E03': // ḃ [LATIN SMALL LETTER B WITH DOT ABOVE] + case '\u1E05': // ḅ [LATIN SMALL LETTER B WITH DOT BELOW] + case '\u1E07': // ḇ [LATIN SMALL LETTER B WITH LINE BELOW] + case '\u24D1': // ⓑ [CIRCLED LATIN SMALL LETTER B] + case '\uFF42': // b [FULLWIDTH LATIN SMALL LETTER B] + output[outputPos++] = 'b'; + break; + case '\u249D': // ⒝ [PARENTHESIZED LATIN SMALL LETTER B] + output[outputPos++] = '('; + output[outputPos++] = 'b'; + output[outputPos++] = ')'; + break; + case '\u00C7': // Ç [LATIN CAPITAL LETTER C WITH CEDILLA] + case '\u0106': // Ć [LATIN CAPITAL LETTER C WITH ACUTE] + case '\u0108': // Ĉ [LATIN CAPITAL LETTER C WITH CIRCUMFLEX] + case '\u010A': // Ċ [LATIN CAPITAL LETTER C WITH DOT ABOVE] + case '\u010C': // Č [LATIN CAPITAL LETTER C WITH CARON] + case '\u0187': // Ƈ [LATIN CAPITAL LETTER C WITH HOOK] + case '\u023B': // Ȼ [LATIN CAPITAL LETTER C WITH STROKE] + case '\u0297': // ʗ [LATIN LETTER STRETCHED C] + case '\u1D04': // ᴄ [LATIN LETTER SMALL CAPITAL C] + case '\u1E08': // Ḉ [LATIN CAPITAL LETTER C WITH CEDILLA AND ACUTE] + case '\u24B8': // Ⓒ [CIRCLED LATIN CAPITAL LETTER C] + case '\uFF23': // C [FULLWIDTH LATIN CAPITAL LETTER C] + output[outputPos++] = 'C'; + break; + case '\u00E7': // ç [LATIN SMALL LETTER C WITH CEDILLA] + case '\u0107': // ć [LATIN SMALL LETTER C WITH ACUTE] + case '\u0109': // ĉ [LATIN SMALL LETTER C WITH CIRCUMFLEX] + case '\u010B': // ċ [LATIN SMALL LETTER C WITH DOT ABOVE] + case '\u010D': // č [LATIN SMALL LETTER C WITH CARON] + case '\u0188': // ƈ [LATIN SMALL LETTER C WITH HOOK] + case '\u023C': // ȼ [LATIN SMALL LETTER C WITH STROKE] + case '\u0255': // ɕ [LATIN SMALL LETTER C WITH CURL] + case '\u1E09': // ḉ [LATIN SMALL LETTER C WITH CEDILLA AND ACUTE] + case '\u2184': // ↄ [LATIN SMALL LETTER REVERSED C] + case '\u24D2': // ⓒ [CIRCLED LATIN SMALL LETTER C] + case '\uA73E': // Ꜿ [LATIN CAPITAL LETTER REVERSED C WITH DOT] + case '\uA73F': // ꜿ [LATIN SMALL LETTER REVERSED C WITH DOT] + case '\uFF43': // c [FULLWIDTH LATIN SMALL LETTER C] + output[outputPos++] = 'c'; + break; + case '\u249E': // ⒞ [PARENTHESIZED LATIN SMALL LETTER C] + output[outputPos++] = '('; + output[outputPos++] = 'c'; + output[outputPos++] = ')'; + break; + case '\u00D0': // Ð [LATIN CAPITAL LETTER ETH] + case '\u010E': // Ď [LATIN CAPITAL LETTER D WITH CARON] + case '\u0110': // Đ [LATIN CAPITAL LETTER D WITH STROKE] + case '\u0189': // Ɖ [LATIN CAPITAL LETTER AFRICAN D] + case '\u018A': // Ɗ [LATIN CAPITAL LETTER D WITH HOOK] + case '\u018B': // Ƌ [LATIN CAPITAL LETTER D WITH TOPBAR] + case '\u1D05': // ᴅ [LATIN LETTER SMALL CAPITAL D] + case '\u1D06': // ᴆ [LATIN LETTER SMALL CAPITAL ETH] + case '\u1E0A': // Ḋ [LATIN CAPITAL LETTER D WITH DOT ABOVE] + case '\u1E0C': // Ḍ [LATIN CAPITAL LETTER D WITH DOT BELOW] + case '\u1E0E': // Ḏ [LATIN CAPITAL LETTER D WITH LINE BELOW] + case '\u1E10': // Ḑ [LATIN CAPITAL LETTER D WITH CEDILLA] + case '\u1E12': // Ḓ [LATIN CAPITAL LETTER D WITH CIRCUMFLEX BELOW] + case '\u24B9': // Ⓓ [CIRCLED LATIN CAPITAL LETTER D] + case '\uA779': // Ꝺ [LATIN CAPITAL LETTER INSULAR D] + case '\uFF24': // D [FULLWIDTH LATIN CAPITAL LETTER D] + output[outputPos++] = 'D'; + break; + case '\u00F0': // ð [LATIN SMALL LETTER ETH] + case '\u010F': // ď [LATIN SMALL LETTER D WITH CARON] + case '\u0111': // đ [LATIN SMALL LETTER D WITH STROKE] + case '\u018C': // ƌ [LATIN SMALL LETTER D WITH TOPBAR] + case '\u0221': // ȡ [LATIN SMALL LETTER D WITH CURL] + case '\u0256': // ɖ [LATIN SMALL LETTER D WITH TAIL] + case '\u0257': // ɗ [LATIN SMALL LETTER D WITH HOOK] + case '\u1D6D': // ᵭ [LATIN SMALL LETTER D WITH MIDDLE TILDE] + case '\u1D81': // ᶁ [LATIN SMALL LETTER D WITH PALATAL HOOK] + case '\u1D91': // ᶑ [LATIN SMALL LETTER D WITH HOOK AND TAIL] + case '\u1E0B': // ḋ [LATIN SMALL LETTER D WITH DOT ABOVE] + case '\u1E0D': // ḍ [LATIN SMALL LETTER D WITH DOT BELOW] + case '\u1E0F': // ḏ [LATIN SMALL LETTER D WITH LINE BELOW] + case '\u1E11': // ḑ [LATIN SMALL LETTER D WITH CEDILLA] + case '\u1E13': // ḓ [LATIN SMALL LETTER D WITH CIRCUMFLEX BELOW] + case '\u24D3': // ⓓ [CIRCLED LATIN SMALL LETTER D] + case '\uA77A': // ꝺ [LATIN SMALL LETTER INSULAR D] + case '\uFF44': // d [FULLWIDTH LATIN SMALL LETTER D] + output[outputPos++] = 'd'; + break; + case '\u01C4': // DŽ [LATIN CAPITAL LETTER DZ WITH CARON] + case '\u01F1': // DZ [LATIN CAPITAL LETTER DZ] + output[outputPos++] = 'D'; + output[outputPos++] = 'Z'; + break; + case '\u01C5': // Dž [LATIN CAPITAL LETTER D WITH SMALL LETTER Z WITH CARON] + case '\u01F2': // Dz [LATIN CAPITAL LETTER D WITH SMALL LETTER Z] + output[outputPos++] = 'D'; + output[outputPos++] = 'z'; + break; + case '\u249F': // ⒟ [PARENTHESIZED LATIN SMALL LETTER D] + output[outputPos++] = '('; + output[outputPos++] = 'd'; + output[outputPos++] = ')'; + break; + case '\u0238': // ȸ [LATIN SMALL LETTER DB DIGRAPH] + output[outputPos++] = 'd'; + output[outputPos++] = 'b'; + break; + case '\u01C6': // dž [LATIN SMALL LETTER DZ WITH CARON] + case '\u01F3': // dz [LATIN SMALL LETTER DZ] + case '\u02A3': // ʣ [LATIN SMALL LETTER DZ DIGRAPH] + case '\u02A5': // ʥ [LATIN SMALL LETTER DZ DIGRAPH WITH CURL] + output[outputPos++] = 'd'; + output[outputPos++] = 'z'; + break; + case '\u00C8': // È [LATIN CAPITAL LETTER E WITH GRAVE] + case '\u00C9': // É [LATIN CAPITAL LETTER E WITH ACUTE] + case '\u00CA': // Ê [LATIN CAPITAL LETTER E WITH CIRCUMFLEX] + case '\u00CB': // Ë [LATIN CAPITAL LETTER E WITH DIAERESIS] + case '\u0112': // Ē [LATIN CAPITAL LETTER E WITH MACRON] + case '\u0114': // Ĕ [LATIN CAPITAL LETTER E WITH BREVE] + case '\u0116': // Ė [LATIN CAPITAL LETTER E WITH DOT ABOVE] + case '\u0118': // Ę [LATIN CAPITAL LETTER E WITH OGONEK] + case '\u011A': // Ě [LATIN CAPITAL LETTER E WITH CARON] + case '\u018E': // Ǝ [LATIN CAPITAL LETTER REVERSED E] + case '\u0190': // Ɛ [LATIN CAPITAL LETTER OPEN E] + case '\u0204': // Ȅ [LATIN CAPITAL LETTER E WITH DOUBLE GRAVE] + case '\u0206': // Ȇ [LATIN CAPITAL LETTER E WITH INVERTED BREVE] + case '\u0228': // Ȩ [LATIN CAPITAL LETTER E WITH CEDILLA] + case '\u0246': // Ɇ [LATIN CAPITAL LETTER E WITH STROKE] + case '\u1D07': // ᴇ [LATIN LETTER SMALL CAPITAL E] + case '\u1E14': // Ḕ [LATIN CAPITAL LETTER E WITH MACRON AND GRAVE] + case '\u1E16': // Ḗ [LATIN CAPITAL LETTER E WITH MACRON AND ACUTE] + case '\u1E18': // Ḙ [LATIN CAPITAL LETTER E WITH CIRCUMFLEX BELOW] + case '\u1E1A': // Ḛ [LATIN CAPITAL LETTER E WITH TILDE BELOW] + case '\u1E1C': // Ḝ [LATIN CAPITAL LETTER E WITH CEDILLA AND BREVE] + case '\u1EB8': // Ẹ [LATIN CAPITAL LETTER E WITH DOT BELOW] + case '\u1EBA': // Ẻ [LATIN CAPITAL LETTER E WITH HOOK ABOVE] + case '\u1EBC': // Ẽ [LATIN CAPITAL LETTER E WITH TILDE] + case '\u1EBE': // Ế [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND ACUTE] + case '\u1EC0': // Ề [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND GRAVE] + case '\u1EC2': // Ể [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND HOOK ABOVE] + case '\u1EC4': // Ễ [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND TILDE] + case '\u1EC6': // Ệ [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND DOT BELOW] + case '\u24BA': // Ⓔ [CIRCLED LATIN CAPITAL LETTER E] + case '\u2C7B': // ⱻ [LATIN LETTER SMALL CAPITAL TURNED E] + case '\uFF25': // E [FULLWIDTH LATIN CAPITAL LETTER E] + output[outputPos++] = 'E'; + break; + case '\u00E8': // è [LATIN SMALL LETTER E WITH GRAVE] + case '\u00E9': // é [LATIN SMALL LETTER E WITH ACUTE] + case '\u00EA': // ê [LATIN SMALL LETTER E WITH CIRCUMFLEX] + case '\u00EB': // ë [LATIN SMALL LETTER E WITH DIAERESIS] + case '\u0113': // ē [LATIN SMALL LETTER E WITH MACRON] + case '\u0115': // ĕ [LATIN SMALL LETTER E WITH BREVE] + case '\u0117': // ė [LATIN SMALL LETTER E WITH DOT ABOVE] + case '\u0119': // ę [LATIN SMALL LETTER E WITH OGONEK] + case '\u011B': // ě [LATIN SMALL LETTER E WITH CARON] + case '\u01DD': // ǝ [LATIN SMALL LETTER TURNED E] + case '\u0205': // ȅ [LATIN SMALL LETTER E WITH DOUBLE GRAVE] + case '\u0207': // ȇ [LATIN SMALL LETTER E WITH INVERTED BREVE] + case '\u0229': // ȩ [LATIN SMALL LETTER E WITH CEDILLA] + case '\u0247': // ɇ [LATIN SMALL LETTER E WITH STROKE] + case '\u0258': // ɘ [LATIN SMALL LETTER REVERSED E] + case '\u025B': // ɛ [LATIN SMALL LETTER OPEN E] + case '\u025C': // ɜ [LATIN SMALL LETTER REVERSED OPEN E] + case '\u025D': // ɝ [LATIN SMALL LETTER REVERSED OPEN E WITH HOOK] + case '\u025E': // ɞ [LATIN SMALL LETTER CLOSED REVERSED OPEN E] + case '\u029A': // ʚ [LATIN SMALL LETTER CLOSED OPEN E] + case '\u1D08': // ᴈ [LATIN SMALL LETTER TURNED OPEN E] + case '\u1D92': // ᶒ [LATIN SMALL LETTER E WITH RETROFLEX HOOK] + case '\u1D93': // ᶓ [LATIN SMALL LETTER OPEN E WITH RETROFLEX HOOK] + case '\u1D94': // ᶔ [LATIN SMALL LETTER REVERSED OPEN E WITH RETROFLEX HOOK] + case '\u1E15': // ḕ [LATIN SMALL LETTER E WITH MACRON AND GRAVE] + case '\u1E17': // ḗ [LATIN SMALL LETTER E WITH MACRON AND ACUTE] + case '\u1E19': // ḙ [LATIN SMALL LETTER E WITH CIRCUMFLEX BELOW] + case '\u1E1B': // ḛ [LATIN SMALL LETTER E WITH TILDE BELOW] + case '\u1E1D': // ḝ [LATIN SMALL LETTER E WITH CEDILLA AND BREVE] + case '\u1EB9': // ẹ [LATIN SMALL LETTER E WITH DOT BELOW] + case '\u1EBB': // ẻ [LATIN SMALL LETTER E WITH HOOK ABOVE] + case '\u1EBD': // ẽ [LATIN SMALL LETTER E WITH TILDE] + case '\u1EBF': // ế [LATIN SMALL LETTER E WITH CIRCUMFLEX AND ACUTE] + case '\u1EC1': // ề [LATIN SMALL LETTER E WITH CIRCUMFLEX AND GRAVE] + case '\u1EC3': // ể [LATIN SMALL LETTER E WITH CIRCUMFLEX AND HOOK ABOVE] + case '\u1EC5': // ễ [LATIN SMALL LETTER E WITH CIRCUMFLEX AND TILDE] + case '\u1EC7': // ệ [LATIN SMALL LETTER E WITH CIRCUMFLEX AND DOT BELOW] + case '\u2091': // ₑ [LATIN SUBSCRIPT SMALL LETTER E] + case '\u24D4': // ⓔ [CIRCLED LATIN SMALL LETTER E] + case '\u2C78': // ⱸ [LATIN SMALL LETTER E WITH NOTCH] + case '\uFF45': // e [FULLWIDTH LATIN SMALL LETTER E] + output[outputPos++] = 'e'; + break; + case '\u24A0': // ⒠ [PARENTHESIZED LATIN SMALL LETTER E] + output[outputPos++] = '('; + output[outputPos++] = 'e'; + output[outputPos++] = ')'; + break; + case '\u0191': // Ƒ [LATIN CAPITAL LETTER F WITH HOOK] + case '\u1E1E': // Ḟ [LATIN CAPITAL LETTER F WITH DOT ABOVE] + case '\u24BB': // Ⓕ [CIRCLED LATIN CAPITAL LETTER F] + case '\uA730': // ꜰ [LATIN LETTER SMALL CAPITAL F] + case '\uA77B': // Ꝼ [LATIN CAPITAL LETTER INSULAR F] + case '\uA7FB': // ꟻ [LATIN EPIGRAPHIC LETTER REVERSED F] + case '\uFF26': // F [FULLWIDTH LATIN CAPITAL LETTER F] + output[outputPos++] = 'F'; + break; + case '\u0192': // ƒ [LATIN SMALL LETTER F WITH HOOK] + case '\u1D6E': // ᵮ [LATIN SMALL LETTER F WITH MIDDLE TILDE] + case '\u1D82': // ᶂ [LATIN SMALL LETTER F WITH PALATAL HOOK] + case '\u1E1F': // ḟ [LATIN SMALL LETTER F WITH DOT ABOVE] + case '\u1E9B': // ẛ [LATIN SMALL LETTER LONG S WITH DOT ABOVE] + case '\u24D5': // ⓕ [CIRCLED LATIN SMALL LETTER F] + case '\uA77C': // ꝼ [LATIN SMALL LETTER INSULAR F] + case '\uFF46': // f [FULLWIDTH LATIN SMALL LETTER F] + output[outputPos++] = 'f'; + break; + case '\u24A1': // ⒡ [PARENTHESIZED LATIN SMALL LETTER F] + output[outputPos++] = '('; + output[outputPos++] = 'f'; + output[outputPos++] = ')'; + break; + case '\uFB00': // ff [LATIN SMALL LIGATURE FF] + output[outputPos++] = 'f'; + output[outputPos++] = 'f'; + break; + case '\uFB03': // ffi [LATIN SMALL LIGATURE FFI] + output[outputPos++] = 'f'; + output[outputPos++] = 'f'; + output[outputPos++] = 'i'; + break; + case '\uFB04': // ffl [LATIN SMALL LIGATURE FFL] + output[outputPos++] = 'f'; + output[outputPos++] = 'f'; + output[outputPos++] = 'l'; + break; + case '\uFB01': // fi [LATIN SMALL LIGATURE FI] + output[outputPos++] = 'f'; + output[outputPos++] = 'i'; + break; + case '\uFB02': // fl [LATIN SMALL LIGATURE FL] + output[outputPos++] = 'f'; + output[outputPos++] = 'l'; + break; + case '\u011C': // Ĝ [LATIN CAPITAL LETTER G WITH CIRCUMFLEX] + case '\u011E': // Ğ [LATIN CAPITAL LETTER G WITH BREVE] + case '\u0120': // Ġ [LATIN CAPITAL LETTER G WITH DOT ABOVE] + case '\u0122': // Ģ [LATIN CAPITAL LETTER G WITH CEDILLA] + case '\u0193': // Ɠ [LATIN CAPITAL LETTER G WITH HOOK] + case '\u01E4': // Ǥ [LATIN CAPITAL LETTER G WITH STROKE] + case '\u01E5': // ǥ [LATIN SMALL LETTER G WITH STROKE] + case '\u01E6': // Ǧ [LATIN CAPITAL LETTER G WITH CARON] + case '\u01E7': // ǧ [LATIN SMALL LETTER G WITH CARON] + case '\u01F4': // Ǵ [LATIN CAPITAL LETTER G WITH ACUTE] + case '\u0262': // ɢ [LATIN LETTER SMALL CAPITAL G] + case '\u029B': // ʛ [LATIN LETTER SMALL CAPITAL G WITH HOOK] + case '\u1E20': // Ḡ [LATIN CAPITAL LETTER G WITH MACRON] + case '\u24BC': // Ⓖ [CIRCLED LATIN CAPITAL LETTER G] + case '\uA77D': // Ᵹ [LATIN CAPITAL LETTER INSULAR G] + case '\uA77E': // Ꝿ [LATIN CAPITAL LETTER TURNED INSULAR G] + case '\uFF27': // G [FULLWIDTH LATIN CAPITAL LETTER G] + output[outputPos++] = 'G'; + break; + case '\u011D': // ĝ [LATIN SMALL LETTER G WITH CIRCUMFLEX] + case '\u011F': // ğ [LATIN SMALL LETTER G WITH BREVE] + case '\u0121': // ġ [LATIN SMALL LETTER G WITH DOT ABOVE] + case '\u0123': // ģ [LATIN SMALL LETTER G WITH CEDILLA] + case '\u01F5': // ǵ [LATIN SMALL LETTER G WITH ACUTE] + case '\u0260': // ɠ [LATIN SMALL LETTER G WITH HOOK] + case '\u0261': // ɡ [LATIN SMALL LETTER SCRIPT G] + case '\u1D77': // ᵷ [LATIN SMALL LETTER TURNED G] + case '\u1D79': // ᵹ [LATIN SMALL LETTER INSULAR G] + case '\u1D83': // ᶃ [LATIN SMALL LETTER G WITH PALATAL HOOK] + case '\u1E21': // ḡ [LATIN SMALL LETTER G WITH MACRON] + case '\u24D6': // ⓖ [CIRCLED LATIN SMALL LETTER G] + case '\uA77F': // ꝿ [LATIN SMALL LETTER TURNED INSULAR G] + case '\uFF47': // g [FULLWIDTH LATIN SMALL LETTER G] + output[outputPos++] = 'g'; + break; + case '\u24A2': // ⒢ [PARENTHESIZED LATIN SMALL LETTER G] + output[outputPos++] = '('; + output[outputPos++] = 'g'; + output[outputPos++] = ')'; + break; + case '\u0124': // Ĥ [LATIN CAPITAL LETTER H WITH CIRCUMFLEX] + case '\u0126': // Ħ [LATIN CAPITAL LETTER H WITH STROKE] + case '\u021E': // Ȟ [LATIN CAPITAL LETTER H WITH CARON] + case '\u029C': // ʜ [LATIN LETTER SMALL CAPITAL H] + case '\u1E22': // Ḣ [LATIN CAPITAL LETTER H WITH DOT ABOVE] + case '\u1E24': // Ḥ [LATIN CAPITAL LETTER H WITH DOT BELOW] + case '\u1E26': // Ḧ [LATIN CAPITAL LETTER H WITH DIAERESIS] + case '\u1E28': // Ḩ [LATIN CAPITAL LETTER H WITH CEDILLA] + case '\u1E2A': // Ḫ [LATIN CAPITAL LETTER H WITH BREVE BELOW] + case '\u24BD': // Ⓗ [CIRCLED LATIN CAPITAL LETTER H] + case '\u2C67': // Ⱨ [LATIN CAPITAL LETTER H WITH DESCENDER] + case '\u2C75': // Ⱶ [LATIN CAPITAL LETTER HALF H] + case '\uFF28': // H [FULLWIDTH LATIN CAPITAL LETTER H] + output[outputPos++] = 'H'; + break; + case '\u0125': // ĥ [LATIN SMALL LETTER H WITH CIRCUMFLEX] + case '\u0127': // ħ [LATIN SMALL LETTER H WITH STROKE] + case '\u021F': // ȟ [LATIN SMALL LETTER H WITH CARON] + case '\u0265': // ɥ [LATIN SMALL LETTER TURNED H] + case '\u0266': // ɦ [LATIN SMALL LETTER H WITH HOOK] + case '\u02AE': // ʮ [LATIN SMALL LETTER TURNED H WITH FISHHOOK] + case '\u02AF': // ʯ [LATIN SMALL LETTER TURNED H WITH FISHHOOK AND TAIL] + case '\u1E23': // ḣ [LATIN SMALL LETTER H WITH DOT ABOVE] + case '\u1E25': // ḥ [LATIN SMALL LETTER H WITH DOT BELOW] + case '\u1E27': // ḧ [LATIN SMALL LETTER H WITH DIAERESIS] + case '\u1E29': // ḩ [LATIN SMALL LETTER H WITH CEDILLA] + case '\u1E2B': // ḫ [LATIN SMALL LETTER H WITH BREVE BELOW] + case '\u1E96': // ẖ [LATIN SMALL LETTER H WITH LINE BELOW] + case '\u24D7': // ⓗ [CIRCLED LATIN SMALL LETTER H] + case '\u2C68': // ⱨ [LATIN SMALL LETTER H WITH DESCENDER] + case '\u2C76': // ⱶ [LATIN SMALL LETTER HALF H] + case '\uFF48': // h [FULLWIDTH LATIN SMALL LETTER H] + output[outputPos++] = 'h'; + break; + case '\u01F6': // Ƕ http://en.wikipedia.org/wiki/Hwair [LATIN CAPITAL LETTER HWAIR] + output[outputPos++] = 'H'; + output[outputPos++] = 'V'; + break; + case '\u24A3': // ⒣ [PARENTHESIZED LATIN SMALL LETTER H] + output[outputPos++] = '('; + output[outputPos++] = 'h'; + output[outputPos++] = ')'; + break; + case '\u0195': // ƕ [LATIN SMALL LETTER HV] + output[outputPos++] = 'h'; + output[outputPos++] = 'v'; + break; + case '\u00CC': // Ì [LATIN CAPITAL LETTER I WITH GRAVE] + case '\u00CD': // Í [LATIN CAPITAL LETTER I WITH ACUTE] + case '\u00CE': // Î [LATIN CAPITAL LETTER I WITH CIRCUMFLEX] + case '\u00CF': // Ï [LATIN CAPITAL LETTER I WITH DIAERESIS] + case '\u0128': // Ĩ [LATIN CAPITAL LETTER I WITH TILDE] + case '\u012A': // Ī [LATIN CAPITAL LETTER I WITH MACRON] + case '\u012C': // Ĭ [LATIN CAPITAL LETTER I WITH BREVE] + case '\u012E': // Į [LATIN CAPITAL LETTER I WITH OGONEK] + case '\u0130': // İ [LATIN CAPITAL LETTER I WITH DOT ABOVE] + case '\u0196': // Ɩ [LATIN CAPITAL LETTER IOTA] + case '\u0197': // Ɨ [LATIN CAPITAL LETTER I WITH STROKE] + case '\u01CF': // Ǐ [LATIN CAPITAL LETTER I WITH CARON] + case '\u0208': // Ȉ [LATIN CAPITAL LETTER I WITH DOUBLE GRAVE] + case '\u020A': // Ȋ [LATIN CAPITAL LETTER I WITH INVERTED BREVE] + case '\u026A': // ɪ [LATIN LETTER SMALL CAPITAL I] + case '\u1D7B': // ᵻ [LATIN SMALL CAPITAL LETTER I WITH STROKE] + case '\u1E2C': // Ḭ [LATIN CAPITAL LETTER I WITH TILDE BELOW] + case '\u1E2E': // Ḯ [LATIN CAPITAL LETTER I WITH DIAERESIS AND ACUTE] + case '\u1EC8': // Ỉ [LATIN CAPITAL LETTER I WITH HOOK ABOVE] + case '\u1ECA': // Ị [LATIN CAPITAL LETTER I WITH DOT BELOW] + case '\u24BE': // Ⓘ [CIRCLED LATIN CAPITAL LETTER I] + case '\uA7FE': // ꟾ [LATIN EPIGRAPHIC LETTER I LONGA] + case '\uFF29': // I [FULLWIDTH LATIN CAPITAL LETTER I] + output[outputPos++] = 'I'; + break; + case '\u00EC': // ì [LATIN SMALL LETTER I WITH GRAVE] + case '\u00ED': // í [LATIN SMALL LETTER I WITH ACUTE] + case '\u00EE': // î [LATIN SMALL LETTER I WITH CIRCUMFLEX] + case '\u00EF': // ï [LATIN SMALL LETTER I WITH DIAERESIS] + case '\u0129': // ĩ [LATIN SMALL LETTER I WITH TILDE] + case '\u012B': // ī [LATIN SMALL LETTER I WITH MACRON] + case '\u012D': // ĭ [LATIN SMALL LETTER I WITH BREVE] + case '\u012F': // į [LATIN SMALL LETTER I WITH OGONEK] + case '\u0131': // ı [LATIN SMALL LETTER DOTLESS I] + case '\u01D0': // ǐ [LATIN SMALL LETTER I WITH CARON] + case '\u0209': // ȉ [LATIN SMALL LETTER I WITH DOUBLE GRAVE] + case '\u020B': // ȋ [LATIN SMALL LETTER I WITH INVERTED BREVE] + case '\u0268': // ɨ [LATIN SMALL LETTER I WITH STROKE] + case '\u1D09': // ᴉ [LATIN SMALL LETTER TURNED I] + case '\u1D62': // ᵢ [LATIN SUBSCRIPT SMALL LETTER I] + case '\u1D7C': // ᵼ [LATIN SMALL LETTER IOTA WITH STROKE] + case '\u1D96': // ᶖ [LATIN SMALL LETTER I WITH RETROFLEX HOOK] + case '\u1E2D': // ḭ [LATIN SMALL LETTER I WITH TILDE BELOW] + case '\u1E2F': // ḯ [LATIN SMALL LETTER I WITH DIAERESIS AND ACUTE] + case '\u1EC9': // ỉ [LATIN SMALL LETTER I WITH HOOK ABOVE] + case '\u1ECB': // ị [LATIN SMALL LETTER I WITH DOT BELOW] + case '\u2071': // ⁱ [SUPERSCRIPT LATIN SMALL LETTER I] + case '\u24D8': // ⓘ [CIRCLED LATIN SMALL LETTER I] + case '\uFF49': // i [FULLWIDTH LATIN SMALL LETTER I] + output[outputPos++] = 'i'; + break; + case '\u0132': // IJ [LATIN CAPITAL LIGATURE IJ] + output[outputPos++] = 'I'; + output[outputPos++] = 'J'; + break; + case '\u24A4': // ⒤ [PARENTHESIZED LATIN SMALL LETTER I] + output[outputPos++] = '('; + output[outputPos++] = 'i'; + output[outputPos++] = ')'; + break; + case '\u0133': // ij [LATIN SMALL LIGATURE IJ] + output[outputPos++] = 'i'; + output[outputPos++] = 'j'; + break; + case '\u0134': // Ĵ [LATIN CAPITAL LETTER J WITH CIRCUMFLEX] + case '\u0248': // Ɉ [LATIN CAPITAL LETTER J WITH STROKE] + case '\u1D0A': // ᴊ [LATIN LETTER SMALL CAPITAL J] + case '\u24BF': // Ⓙ [CIRCLED LATIN CAPITAL LETTER J] + case '\uFF2A': // J [FULLWIDTH LATIN CAPITAL LETTER J] + output[outputPos++] = 'J'; + break; + case '\u0135': // ĵ [LATIN SMALL LETTER J WITH CIRCUMFLEX] + case '\u01F0': // ǰ [LATIN SMALL LETTER J WITH CARON] + case '\u0237': // ȷ [LATIN SMALL LETTER DOTLESS J] + case '\u0249': // ɉ [LATIN SMALL LETTER J WITH STROKE] + case '\u025F': // ɟ [LATIN SMALL LETTER DOTLESS J WITH STROKE] + case '\u0284': // ʄ [LATIN SMALL LETTER DOTLESS J WITH STROKE AND HOOK] + case '\u029D': // ʝ [LATIN SMALL LETTER J WITH CROSSED-TAIL] + case '\u24D9': // ⓙ [CIRCLED LATIN SMALL LETTER J] + case '\u2C7C': // ⱼ [LATIN SUBSCRIPT SMALL LETTER J] + case '\uFF4A': // j [FULLWIDTH LATIN SMALL LETTER J] + output[outputPos++] = 'j'; + break; + case '\u24A5': // ⒥ [PARENTHESIZED LATIN SMALL LETTER J] + output[outputPos++] = '('; + output[outputPos++] = 'j'; + output[outputPos++] = ')'; + break; + case '\u0136': // Ķ [LATIN CAPITAL LETTER K WITH CEDILLA] + case '\u0198': // Ƙ [LATIN CAPITAL LETTER K WITH HOOK] + case '\u01E8': // Ǩ [LATIN CAPITAL LETTER K WITH CARON] + case '\u1D0B': // ᴋ [LATIN LETTER SMALL CAPITAL K] + case '\u1E30': // Ḱ [LATIN CAPITAL LETTER K WITH ACUTE] + case '\u1E32': // Ḳ [LATIN CAPITAL LETTER K WITH DOT BELOW] + case '\u1E34': // Ḵ [LATIN CAPITAL LETTER K WITH LINE BELOW] + case '\u24C0': // Ⓚ [CIRCLED LATIN CAPITAL LETTER K] + case '\u2C69': // Ⱪ [LATIN CAPITAL LETTER K WITH DESCENDER] + case '\uA740': // Ꝁ [LATIN CAPITAL LETTER K WITH STROKE] + case '\uA742': // Ꝃ [LATIN CAPITAL LETTER K WITH DIAGONAL STROKE] + case '\uA744': // Ꝅ [LATIN CAPITAL LETTER K WITH STROKE AND DIAGONAL STROKE] + case '\uFF2B': // K [FULLWIDTH LATIN CAPITAL LETTER K] + output[outputPos++] = 'K'; + break; + case '\u0137': // ķ [LATIN SMALL LETTER K WITH CEDILLA] + case '\u0199': // ƙ [LATIN SMALL LETTER K WITH HOOK] + case '\u01E9': // ǩ [LATIN SMALL LETTER K WITH CARON] + case '\u029E': // ʞ [LATIN SMALL LETTER TURNED K] + case '\u1D84': // ᶄ [LATIN SMALL LETTER K WITH PALATAL HOOK] + case '\u1E31': // ḱ [LATIN SMALL LETTER K WITH ACUTE] + case '\u1E33': // ḳ [LATIN SMALL LETTER K WITH DOT BELOW] + case '\u1E35': // ḵ [LATIN SMALL LETTER K WITH LINE BELOW] + case '\u24DA': // ⓚ [CIRCLED LATIN SMALL LETTER K] + case '\u2C6A': // ⱪ [LATIN SMALL LETTER K WITH DESCENDER] + case '\uA741': // ꝁ [LATIN SMALL LETTER K WITH STROKE] + case '\uA743': // ꝃ [LATIN SMALL LETTER K WITH DIAGONAL STROKE] + case '\uA745': // ꝅ [LATIN SMALL LETTER K WITH STROKE AND DIAGONAL STROKE] + case '\uFF4B': // k [FULLWIDTH LATIN SMALL LETTER K] + output[outputPos++] = 'k'; + break; + case '\u24A6': // ⒦ [PARENTHESIZED LATIN SMALL LETTER K] + output[outputPos++] = '('; + output[outputPos++] = 'k'; + output[outputPos++] = ')'; + break; + case '\u0139': // Ĺ [LATIN CAPITAL LETTER L WITH ACUTE] + case '\u013B': // Ļ [LATIN CAPITAL LETTER L WITH CEDILLA] + case '\u013D': // Ľ [LATIN CAPITAL LETTER L WITH CARON] + case '\u013F': // Ŀ [LATIN CAPITAL LETTER L WITH MIDDLE DOT] + case '\u0141': // Ł [LATIN CAPITAL LETTER L WITH STROKE] + case '\u023D': // Ƚ [LATIN CAPITAL LETTER L WITH BAR] + case '\u029F': // ʟ [LATIN LETTER SMALL CAPITAL L] + case '\u1D0C': // ᴌ [LATIN LETTER SMALL CAPITAL L WITH STROKE] + case '\u1E36': // Ḷ [LATIN CAPITAL LETTER L WITH DOT BELOW] + case '\u1E38': // Ḹ [LATIN CAPITAL LETTER L WITH DOT BELOW AND MACRON] + case '\u1E3A': // Ḻ [LATIN CAPITAL LETTER L WITH LINE BELOW] + case '\u1E3C': // Ḽ [LATIN CAPITAL LETTER L WITH CIRCUMFLEX BELOW] + case '\u24C1': // Ⓛ [CIRCLED LATIN CAPITAL LETTER L] + case '\u2C60': // Ⱡ [LATIN CAPITAL LETTER L WITH DOUBLE BAR] + case '\u2C62': // Ɫ [LATIN CAPITAL LETTER L WITH MIDDLE TILDE] + case '\uA746': // Ꝇ [LATIN CAPITAL LETTER BROKEN L] + case '\uA748': // Ꝉ [LATIN CAPITAL LETTER L WITH HIGH STROKE] + case '\uA780': // Ꞁ [LATIN CAPITAL LETTER TURNED L] + case '\uFF2C': // L [FULLWIDTH LATIN CAPITAL LETTER L] + output[outputPos++] = 'L'; + break; + case '\u013A': // ĺ [LATIN SMALL LETTER L WITH ACUTE] + case '\u013C': // ļ [LATIN SMALL LETTER L WITH CEDILLA] + case '\u013E': // ľ [LATIN SMALL LETTER L WITH CARON] + case '\u0140': // ŀ [LATIN SMALL LETTER L WITH MIDDLE DOT] + case '\u0142': // ł [LATIN SMALL LETTER L WITH STROKE] + case '\u019A': // ƚ [LATIN SMALL LETTER L WITH BAR] + case '\u0234': // ȴ [LATIN SMALL LETTER L WITH CURL] + case '\u026B': // ɫ [LATIN SMALL LETTER L WITH MIDDLE TILDE] + case '\u026C': // ɬ [LATIN SMALL LETTER L WITH BELT] + case '\u026D': // ɭ [LATIN SMALL LETTER L WITH RETROFLEX HOOK] + case '\u1D85': // ᶅ [LATIN SMALL LETTER L WITH PALATAL HOOK] + case '\u1E37': // ḷ [LATIN SMALL LETTER L WITH DOT BELOW] + case '\u1E39': // ḹ [LATIN SMALL LETTER L WITH DOT BELOW AND MACRON] + case '\u1E3B': // ḻ [LATIN SMALL LETTER L WITH LINE BELOW] + case '\u1E3D': // ḽ [LATIN SMALL LETTER L WITH CIRCUMFLEX BELOW] + case '\u24DB': // ⓛ [CIRCLED LATIN SMALL LETTER L] + case '\u2C61': // ⱡ [LATIN SMALL LETTER L WITH DOUBLE BAR] + case '\uA747': // ꝇ [LATIN SMALL LETTER BROKEN L] + case '\uA749': // ꝉ [LATIN SMALL LETTER L WITH HIGH STROKE] + case '\uA781': // ꞁ [LATIN SMALL LETTER TURNED L] + case '\uFF4C': // l [FULLWIDTH LATIN SMALL LETTER L] + output[outputPos++] = 'l'; + break; + case '\u01C7': // LJ [LATIN CAPITAL LETTER LJ] + output[outputPos++] = 'L'; + output[outputPos++] = 'J'; + break; + case '\u1EFA': // Ỻ [LATIN CAPITAL LETTER MIDDLE-WELSH LL] + output[outputPos++] = 'L'; + output[outputPos++] = 'L'; + break; + case '\u01C8': // Lj [LATIN CAPITAL LETTER L WITH SMALL LETTER J] + output[outputPos++] = 'L'; + output[outputPos++] = 'j'; + break; + case '\u24A7': // ⒧ [PARENTHESIZED LATIN SMALL LETTER L] + output[outputPos++] = '('; + output[outputPos++] = 'l'; + output[outputPos++] = ')'; + break; + case '\u01C9': // lj [LATIN SMALL LETTER LJ] + output[outputPos++] = 'l'; + output[outputPos++] = 'j'; + break; + case '\u1EFB': // ỻ [LATIN SMALL LETTER MIDDLE-WELSH LL] + output[outputPos++] = 'l'; + output[outputPos++] = 'l'; + break; + case '\u02AA': // ʪ [LATIN SMALL LETTER LS DIGRAPH] + output[outputPos++] = 'l'; + output[outputPos++] = 's'; + break; + case '\u02AB': // ʫ [LATIN SMALL LETTER LZ DIGRAPH] + output[outputPos++] = 'l'; + output[outputPos++] = 'z'; + break; + case '\u019C': // Ɯ [LATIN CAPITAL LETTER TURNED M] + case '\u1D0D': // ᴍ [LATIN LETTER SMALL CAPITAL M] + case '\u1E3E': // Ḿ [LATIN CAPITAL LETTER M WITH ACUTE] + case '\u1E40': // Ṁ [LATIN CAPITAL LETTER M WITH DOT ABOVE] + case '\u1E42': // Ṃ [LATIN CAPITAL LETTER M WITH DOT BELOW] + case '\u24C2': // Ⓜ [CIRCLED LATIN CAPITAL LETTER M] + case '\u2C6E': // Ɱ [LATIN CAPITAL LETTER M WITH HOOK] + case '\uA7FD': // ꟽ [LATIN EPIGRAPHIC LETTER INVERTED M] + case '\uA7FF': // ꟿ [LATIN EPIGRAPHIC LETTER ARCHAIC M] + case '\uFF2D': // M [FULLWIDTH LATIN CAPITAL LETTER M] + output[outputPos++] = 'M'; + break; + case '\u026F': // ɯ [LATIN SMALL LETTER TURNED M] + case '\u0270': // ɰ [LATIN SMALL LETTER TURNED M WITH LONG LEG] + case '\u0271': // ɱ [LATIN SMALL LETTER M WITH HOOK] + case '\u1D6F': // ᵯ [LATIN SMALL LETTER M WITH MIDDLE TILDE] + case '\u1D86': // ᶆ [LATIN SMALL LETTER M WITH PALATAL HOOK] + case '\u1E3F': // ḿ [LATIN SMALL LETTER M WITH ACUTE] + case '\u1E41': // ṁ [LATIN SMALL LETTER M WITH DOT ABOVE] + case '\u1E43': // ṃ [LATIN SMALL LETTER M WITH DOT BELOW] + case '\u24DC': // ⓜ [CIRCLED LATIN SMALL LETTER M] + case '\uFF4D': // m [FULLWIDTH LATIN SMALL LETTER M] + output[outputPos++] = 'm'; + break; + case '\u24A8': // ⒨ [PARENTHESIZED LATIN SMALL LETTER M] + output[outputPos++] = '('; + output[outputPos++] = 'm'; + output[outputPos++] = ')'; + break; + case '\u00D1': // Ñ [LATIN CAPITAL LETTER N WITH TILDE] + case '\u0143': // Ń [LATIN CAPITAL LETTER N WITH ACUTE] + case '\u0145': // Ņ [LATIN CAPITAL LETTER N WITH CEDILLA] + case '\u0147': // Ň [LATIN CAPITAL LETTER N WITH CARON] + case '\u014A': // Ŋ http://en.wikipedia.org/wiki/Eng_(letter) [LATIN CAPITAL LETTER ENG] + case '\u019D': // Ɲ [LATIN CAPITAL LETTER N WITH LEFT HOOK] + case '\u01F8': // Ǹ [LATIN CAPITAL LETTER N WITH GRAVE] + case '\u0220': // Ƞ [LATIN CAPITAL LETTER N WITH LONG RIGHT LEG] + case '\u0274': // ɴ [LATIN LETTER SMALL CAPITAL N] + case '\u1D0E': // ᴎ [LATIN LETTER SMALL CAPITAL REVERSED N] + case '\u1E44': // Ṅ [LATIN CAPITAL LETTER N WITH DOT ABOVE] + case '\u1E46': // Ṇ [LATIN CAPITAL LETTER N WITH DOT BELOW] + case '\u1E48': // Ṉ [LATIN CAPITAL LETTER N WITH LINE BELOW] + case '\u1E4A': // Ṋ [LATIN CAPITAL LETTER N WITH CIRCUMFLEX BELOW] + case '\u24C3': // Ⓝ [CIRCLED LATIN CAPITAL LETTER N] + case '\uFF2E': // N [FULLWIDTH LATIN CAPITAL LETTER N] + output[outputPos++] = 'N'; + break; + case '\u00F1': // ñ [LATIN SMALL LETTER N WITH TILDE] + case '\u0144': // ń [LATIN SMALL LETTER N WITH ACUTE] + case '\u0146': // ņ [LATIN SMALL LETTER N WITH CEDILLA] + case '\u0148': // ň [LATIN SMALL LETTER N WITH CARON] + case '\u0149': // ʼn [LATIN SMALL LETTER N PRECEDED BY APOSTROPHE] + case '\u014B': // ŋ http://en.wikipedia.org/wiki/Eng_(letter) [LATIN SMALL LETTER ENG] + case '\u019E': // ƞ [LATIN SMALL LETTER N WITH LONG RIGHT LEG] + case '\u01F9': // ǹ [LATIN SMALL LETTER N WITH GRAVE] + case '\u0235': // ȵ [LATIN SMALL LETTER N WITH CURL] + case '\u0272': // ɲ [LATIN SMALL LETTER N WITH LEFT HOOK] + case '\u0273': // ɳ [LATIN SMALL LETTER N WITH RETROFLEX HOOK] + case '\u1D70': // ᵰ [LATIN SMALL LETTER N WITH MIDDLE TILDE] + case '\u1D87': // ᶇ [LATIN SMALL LETTER N WITH PALATAL HOOK] + case '\u1E45': // ṅ [LATIN SMALL LETTER N WITH DOT ABOVE] + case '\u1E47': // ṇ [LATIN SMALL LETTER N WITH DOT BELOW] + case '\u1E49': // ṉ [LATIN SMALL LETTER N WITH LINE BELOW] + case '\u1E4B': // ṋ [LATIN SMALL LETTER N WITH CIRCUMFLEX BELOW] + case '\u207F': // ⁿ [SUPERSCRIPT LATIN SMALL LETTER N] + case '\u24DD': // ⓝ [CIRCLED LATIN SMALL LETTER N] + case '\uFF4E': // n [FULLWIDTH LATIN SMALL LETTER N] + output[outputPos++] = 'n'; + break; + case '\u01CA': // NJ [LATIN CAPITAL LETTER NJ] + output[outputPos++] = 'N'; + output[outputPos++] = 'J'; + break; + case '\u01CB': // Nj [LATIN CAPITAL LETTER N WITH SMALL LETTER J] + output[outputPos++] = 'N'; + output[outputPos++] = 'j'; + break; + case '\u24A9': // ⒩ [PARENTHESIZED LATIN SMALL LETTER N] + output[outputPos++] = '('; + output[outputPos++] = 'n'; + output[outputPos++] = ')'; + break; + case '\u01CC': // nj [LATIN SMALL LETTER NJ] + output[outputPos++] = 'n'; + output[outputPos++] = 'j'; + break; + case '\u00D2': // Ò [LATIN CAPITAL LETTER O WITH GRAVE] + case '\u00D3': // Ó [LATIN CAPITAL LETTER O WITH ACUTE] + case '\u00D4': // Ô [LATIN CAPITAL LETTER O WITH CIRCUMFLEX] + case '\u00D5': // Õ [LATIN CAPITAL LETTER O WITH TILDE] + case '\u00D6': // Ö [LATIN CAPITAL LETTER O WITH DIAERESIS] + case '\u00D8': // Ø [LATIN CAPITAL LETTER O WITH STROKE] + case '\u014C': // Ō [LATIN CAPITAL LETTER O WITH MACRON] + case '\u014E': // Ŏ [LATIN CAPITAL LETTER O WITH BREVE] + case '\u0150': // Ő [LATIN CAPITAL LETTER O WITH DOUBLE ACUTE] + case '\u0186': // Ɔ [LATIN CAPITAL LETTER OPEN O] + case '\u019F': // Ɵ [LATIN CAPITAL LETTER O WITH MIDDLE TILDE] + case '\u01A0': // Ơ [LATIN CAPITAL LETTER O WITH HORN] + case '\u01D1': // Ǒ [LATIN CAPITAL LETTER O WITH CARON] + case '\u01EA': // Ǫ [LATIN CAPITAL LETTER O WITH OGONEK] + case '\u01EC': // Ǭ [LATIN CAPITAL LETTER O WITH OGONEK AND MACRON] + case '\u01FE': // Ǿ [LATIN CAPITAL LETTER O WITH STROKE AND ACUTE] + case '\u020C': // Ȍ [LATIN CAPITAL LETTER O WITH DOUBLE GRAVE] + case '\u020E': // Ȏ [LATIN CAPITAL LETTER O WITH INVERTED BREVE] + case '\u022A': // Ȫ [LATIN CAPITAL LETTER O WITH DIAERESIS AND MACRON] + case '\u022C': // Ȭ [LATIN CAPITAL LETTER O WITH TILDE AND MACRON] + case '\u022E': // Ȯ [LATIN CAPITAL LETTER O WITH DOT ABOVE] + case '\u0230': // Ȱ [LATIN CAPITAL LETTER O WITH DOT ABOVE AND MACRON] + case '\u1D0F': // ᴏ [LATIN LETTER SMALL CAPITAL O] + case '\u1D10': // ᴐ [LATIN LETTER SMALL CAPITAL OPEN O] + case '\u1E4C': // Ṍ [LATIN CAPITAL LETTER O WITH TILDE AND ACUTE] + case '\u1E4E': // Ṏ [LATIN CAPITAL LETTER O WITH TILDE AND DIAERESIS] + case '\u1E50': // Ṑ [LATIN CAPITAL LETTER O WITH MACRON AND GRAVE] + case '\u1E52': // Ṓ [LATIN CAPITAL LETTER O WITH MACRON AND ACUTE] + case '\u1ECC': // Ọ [LATIN CAPITAL LETTER O WITH DOT BELOW] + case '\u1ECE': // Ỏ [LATIN CAPITAL LETTER O WITH HOOK ABOVE] + case '\u1ED0': // Ố [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND ACUTE] + case '\u1ED2': // Ồ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND GRAVE] + case '\u1ED4': // Ổ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND HOOK ABOVE] + case '\u1ED6': // Ỗ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND TILDE] + case '\u1ED8': // Ộ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND DOT BELOW] + case '\u1EDA': // Ớ [LATIN CAPITAL LETTER O WITH HORN AND ACUTE] + case '\u1EDC': // Ờ [LATIN CAPITAL LETTER O WITH HORN AND GRAVE] + case '\u1EDE': // Ở [LATIN CAPITAL LETTER O WITH HORN AND HOOK ABOVE] + case '\u1EE0': // Ỡ [LATIN CAPITAL LETTER O WITH HORN AND TILDE] + case '\u1EE2': // Ợ [LATIN CAPITAL LETTER O WITH HORN AND DOT BELOW] + case '\u24C4': // Ⓞ [CIRCLED LATIN CAPITAL LETTER O] + case '\uA74A': // Ꝋ [LATIN CAPITAL LETTER O WITH LONG STROKE OVERLAY] + case '\uA74C': // Ꝍ [LATIN CAPITAL LETTER O WITH LOOP] + case '\uFF2F': // O [FULLWIDTH LATIN CAPITAL LETTER O] + output[outputPos++] = 'O'; + break; + case '\u00F2': // ò [LATIN SMALL LETTER O WITH GRAVE] + case '\u00F3': // ó [LATIN SMALL LETTER O WITH ACUTE] + case '\u00F4': // ô [LATIN SMALL LETTER O WITH CIRCUMFLEX] + case '\u00F5': // õ [LATIN SMALL LETTER O WITH TILDE] + case '\u00F6': // ö [LATIN SMALL LETTER O WITH DIAERESIS] + case '\u00F8': // ø [LATIN SMALL LETTER O WITH STROKE] + case '\u014D': // ō [LATIN SMALL LETTER O WITH MACRON] + case '\u014F': // ŏ [LATIN SMALL LETTER O WITH BREVE] + case '\u0151': // ő [LATIN SMALL LETTER O WITH DOUBLE ACUTE] + case '\u01A1': // ơ [LATIN SMALL LETTER O WITH HORN] + case '\u01D2': // ǒ [LATIN SMALL LETTER O WITH CARON] + case '\u01EB': // ǫ [LATIN SMALL LETTER O WITH OGONEK] + case '\u01ED': // ǭ [LATIN SMALL LETTER O WITH OGONEK AND MACRON] + case '\u01FF': // ǿ [LATIN SMALL LETTER O WITH STROKE AND ACUTE] + case '\u020D': // ȍ [LATIN SMALL LETTER O WITH DOUBLE GRAVE] + case '\u020F': // ȏ [LATIN SMALL LETTER O WITH INVERTED BREVE] + case '\u022B': // ȫ [LATIN SMALL LETTER O WITH DIAERESIS AND MACRON] + case '\u022D': // ȭ [LATIN SMALL LETTER O WITH TILDE AND MACRON] + case '\u022F': // ȯ [LATIN SMALL LETTER O WITH DOT ABOVE] + case '\u0231': // ȱ [LATIN SMALL LETTER O WITH DOT ABOVE AND MACRON] + case '\u0254': // ɔ [LATIN SMALL LETTER OPEN O] + case '\u0275': // ɵ [LATIN SMALL LETTER BARRED O] + case '\u1D16': // ᴖ [LATIN SMALL LETTER TOP HALF O] + case '\u1D17': // ᴗ [LATIN SMALL LETTER BOTTOM HALF O] + case '\u1D97': // ᶗ [LATIN SMALL LETTER OPEN O WITH RETROFLEX HOOK] + case '\u1E4D': // ṍ [LATIN SMALL LETTER O WITH TILDE AND ACUTE] + case '\u1E4F': // ṏ [LATIN SMALL LETTER O WITH TILDE AND DIAERESIS] + case '\u1E51': // ṑ [LATIN SMALL LETTER O WITH MACRON AND GRAVE] + case '\u1E53': // ṓ [LATIN SMALL LETTER O WITH MACRON AND ACUTE] + case '\u1ECD': // ọ [LATIN SMALL LETTER O WITH DOT BELOW] + case '\u1ECF': // ỏ [LATIN SMALL LETTER O WITH HOOK ABOVE] + case '\u1ED1': // ố [LATIN SMALL LETTER O WITH CIRCUMFLEX AND ACUTE] + case '\u1ED3': // ồ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND GRAVE] + case '\u1ED5': // ổ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND HOOK ABOVE] + case '\u1ED7': // ỗ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND TILDE] + case '\u1ED9': // ộ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND DOT BELOW] + case '\u1EDB': // ớ [LATIN SMALL LETTER O WITH HORN AND ACUTE] + case '\u1EDD': // ờ [LATIN SMALL LETTER O WITH HORN AND GRAVE] + case '\u1EDF': // ở [LATIN SMALL LETTER O WITH HORN AND HOOK ABOVE] + case '\u1EE1': // ỡ [LATIN SMALL LETTER O WITH HORN AND TILDE] + case '\u1EE3': // ợ [LATIN SMALL LETTER O WITH HORN AND DOT BELOW] + case '\u2092': // ₒ [LATIN SUBSCRIPT SMALL LETTER O] + case '\u24DE': // ⓞ [CIRCLED LATIN SMALL LETTER O] + case '\u2C7A': // ⱺ [LATIN SMALL LETTER O WITH LOW RING INSIDE] + case '\uA74B': // ꝋ [LATIN SMALL LETTER O WITH LONG STROKE OVERLAY] + case '\uA74D': // ꝍ [LATIN SMALL LETTER O WITH LOOP] + case '\uFF4F': // o [FULLWIDTH LATIN SMALL LETTER O] + output[outputPos++] = 'o'; + break; + case '\u0152': // Œ [LATIN CAPITAL LIGATURE OE] + case '\u0276': // ɶ [LATIN LETTER SMALL CAPITAL OE] + output[outputPos++] = 'O'; + output[outputPos++] = 'E'; + break; + case '\uA74E': // Ꝏ [LATIN CAPITAL LETTER OO] + output[outputPos++] = 'O'; + output[outputPos++] = 'O'; + break; + case '\u0222': // Ȣ http://en.wikipedia.org/wiki/OU [LATIN CAPITAL LETTER OU] + case '\u1D15': // ᴕ [LATIN LETTER SMALL CAPITAL OU] + output[outputPos++] = 'O'; + output[outputPos++] = 'U'; + break; + case '\u24AA': // ⒪ [PARENTHESIZED LATIN SMALL LETTER O] + output[outputPos++] = '('; + output[outputPos++] = 'o'; + output[outputPos++] = ')'; + break; + case '\u0153': // œ [LATIN SMALL LIGATURE OE] + case '\u1D14': // ᴔ [LATIN SMALL LETTER TURNED OE] + output[outputPos++] = 'o'; + output[outputPos++] = 'e'; + break; + case '\uA74F': // ꝏ [LATIN SMALL LETTER OO] + output[outputPos++] = 'o'; + output[outputPos++] = 'o'; + break; + case '\u0223': // ȣ http://en.wikipedia.org/wiki/OU [LATIN SMALL LETTER OU] + output[outputPos++] = 'o'; + output[outputPos++] = 'u'; + break; + case '\u01A4': // Ƥ [LATIN CAPITAL LETTER P WITH HOOK] + case '\u1D18': // ᴘ [LATIN LETTER SMALL CAPITAL P] + case '\u1E54': // Ṕ [LATIN CAPITAL LETTER P WITH ACUTE] + case '\u1E56': // Ṗ [LATIN CAPITAL LETTER P WITH DOT ABOVE] + case '\u24C5': // Ⓟ [CIRCLED LATIN CAPITAL LETTER P] + case '\u2C63': // Ᵽ [LATIN CAPITAL LETTER P WITH STROKE] + case '\uA750': // Ꝑ [LATIN CAPITAL LETTER P WITH STROKE THROUGH DESCENDER] + case '\uA752': // Ꝓ [LATIN CAPITAL LETTER P WITH FLOURISH] + case '\uA754': // Ꝕ [LATIN CAPITAL LETTER P WITH SQUIRREL TAIL] + case '\uFF30': // P [FULLWIDTH LATIN CAPITAL LETTER P] + output[outputPos++] = 'P'; + break; + case '\u01A5': // ƥ [LATIN SMALL LETTER P WITH HOOK] + case '\u1D71': // ᵱ [LATIN SMALL LETTER P WITH MIDDLE TILDE] + case '\u1D7D': // ᵽ [LATIN SMALL LETTER P WITH STROKE] + case '\u1D88': // ᶈ [LATIN SMALL LETTER P WITH PALATAL HOOK] + case '\u1E55': // ṕ [LATIN SMALL LETTER P WITH ACUTE] + case '\u1E57': // ṗ [LATIN SMALL LETTER P WITH DOT ABOVE] + case '\u24DF': // ⓟ [CIRCLED LATIN SMALL LETTER P] + case '\uA751': // ꝑ [LATIN SMALL LETTER P WITH STROKE THROUGH DESCENDER] + case '\uA753': // ꝓ [LATIN SMALL LETTER P WITH FLOURISH] + case '\uA755': // ꝕ [LATIN SMALL LETTER P WITH SQUIRREL TAIL] + case '\uA7FC': // ꟼ [LATIN EPIGRAPHIC LETTER REVERSED P] + case '\uFF50': // p [FULLWIDTH LATIN SMALL LETTER P] + output[outputPos++] = 'p'; + break; + case '\u24AB': // ⒫ [PARENTHESIZED LATIN SMALL LETTER P] + output[outputPos++] = '('; + output[outputPos++] = 'p'; + output[outputPos++] = ')'; + break; + case '\u024A': // Ɋ [LATIN CAPITAL LETTER SMALL Q WITH HOOK TAIL] + case '\u24C6': // Ⓠ [CIRCLED LATIN CAPITAL LETTER Q] + case '\uA756': // Ꝗ [LATIN CAPITAL LETTER Q WITH STROKE THROUGH DESCENDER] + case '\uA758': // Ꝙ [LATIN CAPITAL LETTER Q WITH DIAGONAL STROKE] + case '\uFF31': // Q [FULLWIDTH LATIN CAPITAL LETTER Q] + output[outputPos++] = 'Q'; + break; + case '\u0138': // ĸ http://en.wikipedia.org/wiki/Kra_(letter) [LATIN SMALL LETTER KRA] + case '\u024B': // ɋ [LATIN SMALL LETTER Q WITH HOOK TAIL] + case '\u02A0': // ʠ [LATIN SMALL LETTER Q WITH HOOK] + case '\u24E0': // ⓠ [CIRCLED LATIN SMALL LETTER Q] + case '\uA757': // ꝗ [LATIN SMALL LETTER Q WITH STROKE THROUGH DESCENDER] + case '\uA759': // ꝙ [LATIN SMALL LETTER Q WITH DIAGONAL STROKE] + case '\uFF51': // q [FULLWIDTH LATIN SMALL LETTER Q] + output[outputPos++] = 'q'; + break; + case '\u24AC': // ⒬ [PARENTHESIZED LATIN SMALL LETTER Q] + output[outputPos++] = '('; + output[outputPos++] = 'q'; + output[outputPos++] = ')'; + break; + case '\u0239': // ȹ [LATIN SMALL LETTER QP DIGRAPH] + output[outputPos++] = 'q'; + output[outputPos++] = 'p'; + break; + case '\u0154': // Ŕ [LATIN CAPITAL LETTER R WITH ACUTE] + case '\u0156': // Ŗ [LATIN CAPITAL LETTER R WITH CEDILLA] + case '\u0158': // Ř [LATIN CAPITAL LETTER R WITH CARON] + case '\u0210': // Ȓ [LATIN CAPITAL LETTER R WITH DOUBLE GRAVE] + case '\u0212': // Ȓ [LATIN CAPITAL LETTER R WITH INVERTED BREVE] + case '\u024C': // Ɍ [LATIN CAPITAL LETTER R WITH STROKE] + case '\u0280': // ʀ [LATIN LETTER SMALL CAPITAL R] + case '\u0281': // ʁ [LATIN LETTER SMALL CAPITAL INVERTED R] + case '\u1D19': // ᴙ [LATIN LETTER SMALL CAPITAL REVERSED R] + case '\u1D1A': // ᴚ [LATIN LETTER SMALL CAPITAL TURNED R] + case '\u1E58': // Ṙ [LATIN CAPITAL LETTER R WITH DOT ABOVE] + case '\u1E5A': // Ṛ [LATIN CAPITAL LETTER R WITH DOT BELOW] + case '\u1E5C': // Ṝ [LATIN CAPITAL LETTER R WITH DOT BELOW AND MACRON] + case '\u1E5E': // Ṟ [LATIN CAPITAL LETTER R WITH LINE BELOW] + case '\u24C7': // Ⓡ [CIRCLED LATIN CAPITAL LETTER R] + case '\u2C64': // Ɽ [LATIN CAPITAL LETTER R WITH TAIL] + case '\uA75A': // Ꝛ [LATIN CAPITAL LETTER R ROTUNDA] + case '\uA782': // Ꞃ [LATIN CAPITAL LETTER INSULAR R] + case '\uFF32': // R [FULLWIDTH LATIN CAPITAL LETTER R] + output[outputPos++] = 'R'; + break; + case '\u0155': // ŕ [LATIN SMALL LETTER R WITH ACUTE] + case '\u0157': // ŗ [LATIN SMALL LETTER R WITH CEDILLA] + case '\u0159': // ř [LATIN SMALL LETTER R WITH CARON] + case '\u0211': // ȑ [LATIN SMALL LETTER R WITH DOUBLE GRAVE] + case '\u0213': // ȓ [LATIN SMALL LETTER R WITH INVERTED BREVE] + case '\u024D': // ɍ [LATIN SMALL LETTER R WITH STROKE] + case '\u027C': // ɼ [LATIN SMALL LETTER R WITH LONG LEG] + case '\u027D': // ɽ [LATIN SMALL LETTER R WITH TAIL] + case '\u027E': // ɾ [LATIN SMALL LETTER R WITH FISHHOOK] + case '\u027F': // ɿ [LATIN SMALL LETTER REVERSED R WITH FISHHOOK] + case '\u1D63': // ᵣ [LATIN SUBSCRIPT SMALL LETTER R] + case '\u1D72': // ᵲ [LATIN SMALL LETTER R WITH MIDDLE TILDE] + case '\u1D73': // ᵳ [LATIN SMALL LETTER R WITH FISHHOOK AND MIDDLE TILDE] + case '\u1D89': // ᶉ [LATIN SMALL LETTER R WITH PALATAL HOOK] + case '\u1E59': // ṙ [LATIN SMALL LETTER R WITH DOT ABOVE] + case '\u1E5B': // ṛ [LATIN SMALL LETTER R WITH DOT BELOW] + case '\u1E5D': // ṝ [LATIN SMALL LETTER R WITH DOT BELOW AND MACRON] + case '\u1E5F': // ṟ [LATIN SMALL LETTER R WITH LINE BELOW] + case '\u24E1': // ⓡ [CIRCLED LATIN SMALL LETTER R] + case '\uA75B': // ꝛ [LATIN SMALL LETTER R ROTUNDA] + case '\uA783': // ꞃ [LATIN SMALL LETTER INSULAR R] + case '\uFF52': // r [FULLWIDTH LATIN SMALL LETTER R] + output[outputPos++] = 'r'; + break; + case '\u24AD': // ⒭ [PARENTHESIZED LATIN SMALL LETTER R] + output[outputPos++] = '('; + output[outputPos++] = 'r'; + output[outputPos++] = ')'; + break; + case '\u015A': // Ś [LATIN CAPITAL LETTER S WITH ACUTE] + case '\u015C': // Ŝ [LATIN CAPITAL LETTER S WITH CIRCUMFLEX] + case '\u015E': // Ş [LATIN CAPITAL LETTER S WITH CEDILLA] + case '\u0160': // Š [LATIN CAPITAL LETTER S WITH CARON] + case '\u0218': // Ș [LATIN CAPITAL LETTER S WITH COMMA BELOW] + case '\u1E60': // Ṡ [LATIN CAPITAL LETTER S WITH DOT ABOVE] + case '\u1E62': // Ṣ [LATIN CAPITAL LETTER S WITH DOT BELOW] + case '\u1E64': // Ṥ [LATIN CAPITAL LETTER S WITH ACUTE AND DOT ABOVE] + case '\u1E66': // Ṧ [LATIN CAPITAL LETTER S WITH CARON AND DOT ABOVE] + case '\u1E68': // Ṩ [LATIN CAPITAL LETTER S WITH DOT BELOW AND DOT ABOVE] + case '\u24C8': // Ⓢ [CIRCLED LATIN CAPITAL LETTER S] + case '\uA731': // ꜱ [LATIN LETTER SMALL CAPITAL S] + case '\uA785': // ꞅ [LATIN SMALL LETTER INSULAR S] + case '\uFF33': // S [FULLWIDTH LATIN CAPITAL LETTER S] + output[outputPos++] = 'S'; + break; + case '\u015B': // ś [LATIN SMALL LETTER S WITH ACUTE] + case '\u015D': // ŝ [LATIN SMALL LETTER S WITH CIRCUMFLEX] + case '\u015F': // ş [LATIN SMALL LETTER S WITH CEDILLA] + case '\u0161': // š [LATIN SMALL LETTER S WITH CARON] + case '\u017F': // ſ http://en.wikipedia.org/wiki/Long_S [LATIN SMALL LETTER LONG S] + case '\u0219': // ș [LATIN SMALL LETTER S WITH COMMA BELOW] + case '\u023F': // ȿ [LATIN SMALL LETTER S WITH SWASH TAIL] + case '\u0282': // ʂ [LATIN SMALL LETTER S WITH HOOK] + case '\u1D74': // ᵴ [LATIN SMALL LETTER S WITH MIDDLE TILDE] + case '\u1D8A': // ᶊ [LATIN SMALL LETTER S WITH PALATAL HOOK] + case '\u1E61': // ṡ [LATIN SMALL LETTER S WITH DOT ABOVE] + case '\u1E63': // ṣ [LATIN SMALL LETTER S WITH DOT BELOW] + case '\u1E65': // ṥ [LATIN SMALL LETTER S WITH ACUTE AND DOT ABOVE] + case '\u1E67': // ṧ [LATIN SMALL LETTER S WITH CARON AND DOT ABOVE] + case '\u1E69': // ṩ [LATIN SMALL LETTER S WITH DOT BELOW AND DOT ABOVE] + case '\u1E9C': // ẜ [LATIN SMALL LETTER LONG S WITH DIAGONAL STROKE] + case '\u1E9D': // ẝ [LATIN SMALL LETTER LONG S WITH HIGH STROKE] + case '\u24E2': // ⓢ [CIRCLED LATIN SMALL LETTER S] + case '\uA784': // Ꞅ [LATIN CAPITAL LETTER INSULAR S] + case '\uFF53': // s [FULLWIDTH LATIN SMALL LETTER S] + output[outputPos++] = 's'; + break; + case '\u1E9E': // ẞ [LATIN CAPITAL LETTER SHARP S] + output[outputPos++] = 'S'; + output[outputPos++] = 'S'; + break; + case '\u24AE': // ⒮ [PARENTHESIZED LATIN SMALL LETTER S] + output[outputPos++] = '('; + output[outputPos++] = 's'; + output[outputPos++] = ')'; + break; + case '\u00DF': // ß [LATIN SMALL LETTER SHARP S] + output[outputPos++] = 's'; + output[outputPos++] = 's'; + break; + case '\uFB06': // st [LATIN SMALL LIGATURE ST] + output[outputPos++] = 's'; + output[outputPos++] = 't'; + break; + case '\u0162': // Ţ [LATIN CAPITAL LETTER T WITH CEDILLA] + case '\u0164': // Ť [LATIN CAPITAL LETTER T WITH CARON] + case '\u0166': // Ŧ [LATIN CAPITAL LETTER T WITH STROKE] + case '\u01AC': // Ƭ [LATIN CAPITAL LETTER T WITH HOOK] + case '\u01AE': // Ʈ [LATIN CAPITAL LETTER T WITH RETROFLEX HOOK] + case '\u021A': // Ț [LATIN CAPITAL LETTER T WITH COMMA BELOW] + case '\u023E': // Ⱦ [LATIN CAPITAL LETTER T WITH DIAGONAL STROKE] + case '\u1D1B': // ᴛ [LATIN LETTER SMALL CAPITAL T] + case '\u1E6A': // Ṫ [LATIN CAPITAL LETTER T WITH DOT ABOVE] + case '\u1E6C': // Ṭ [LATIN CAPITAL LETTER T WITH DOT BELOW] + case '\u1E6E': // Ṯ [LATIN CAPITAL LETTER T WITH LINE BELOW] + case '\u1E70': // Ṱ [LATIN CAPITAL LETTER T WITH CIRCUMFLEX BELOW] + case '\u24C9': // Ⓣ [CIRCLED LATIN CAPITAL LETTER T] + case '\uA786': // Ꞇ [LATIN CAPITAL LETTER INSULAR T] + case '\uFF34': // T [FULLWIDTH LATIN CAPITAL LETTER T] + output[outputPos++] = 'T'; + break; + case '\u0163': // ţ [LATIN SMALL LETTER T WITH CEDILLA] + case '\u0165': // ť [LATIN SMALL LETTER T WITH CARON] + case '\u0167': // ŧ [LATIN SMALL LETTER T WITH STROKE] + case '\u01AB': // ƫ [LATIN SMALL LETTER T WITH PALATAL HOOK] + case '\u01AD': // ƭ [LATIN SMALL LETTER T WITH HOOK] + case '\u021B': // ț [LATIN SMALL LETTER T WITH COMMA BELOW] + case '\u0236': // ȶ [LATIN SMALL LETTER T WITH CURL] + case '\u0287': // ʇ [LATIN SMALL LETTER TURNED T] + case '\u0288': // ʈ [LATIN SMALL LETTER T WITH RETROFLEX HOOK] + case '\u1D75': // ᵵ [LATIN SMALL LETTER T WITH MIDDLE TILDE] + case '\u1E6B': // ṫ [LATIN SMALL LETTER T WITH DOT ABOVE] + case '\u1E6D': // ṭ [LATIN SMALL LETTER T WITH DOT BELOW] + case '\u1E6F': // ṯ [LATIN SMALL LETTER T WITH LINE BELOW] + case '\u1E71': // ṱ [LATIN SMALL LETTER T WITH CIRCUMFLEX BELOW] + case '\u1E97': // ẗ [LATIN SMALL LETTER T WITH DIAERESIS] + case '\u24E3': // ⓣ [CIRCLED LATIN SMALL LETTER T] + case '\u2C66': // ⱦ [LATIN SMALL LETTER T WITH DIAGONAL STROKE] + case '\uFF54': // t [FULLWIDTH LATIN SMALL LETTER T] + output[outputPos++] = 't'; + break; + case '\u00DE': // Þ [LATIN CAPITAL LETTER THORN] + case '\uA766': // Ꝧ [LATIN CAPITAL LETTER THORN WITH STROKE THROUGH DESCENDER] + output[outputPos++] = 'T'; + output[outputPos++] = 'H'; + break; + case '\uA728': // Ꜩ [LATIN CAPITAL LETTER TZ] + output[outputPos++] = 'T'; + output[outputPos++] = 'Z'; + break; + case '\u24AF': // ⒯ [PARENTHESIZED LATIN SMALL LETTER T] + output[outputPos++] = '('; + output[outputPos++] = 't'; + output[outputPos++] = ')'; + break; + case '\u02A8': // ʨ [LATIN SMALL LETTER TC DIGRAPH WITH CURL] + output[outputPos++] = 't'; + output[outputPos++] = 'c'; + break; + case '\u00FE': // þ [LATIN SMALL LETTER THORN] + case '\u1D7A': // ᵺ [LATIN SMALL LETTER TH WITH STRIKETHROUGH] + case '\uA767': // ꝧ [LATIN SMALL LETTER THORN WITH STROKE THROUGH DESCENDER] + output[outputPos++] = 't'; + output[outputPos++] = 'h'; + break; + case '\u02A6': // ʦ [LATIN SMALL LETTER TS DIGRAPH] + output[outputPos++] = 't'; + output[outputPos++] = 's'; + break; + case '\uA729': // ꜩ [LATIN SMALL LETTER TZ] + output[outputPos++] = 't'; + output[outputPos++] = 'z'; + break; + case '\u00D9': // Ù [LATIN CAPITAL LETTER U WITH GRAVE] + case '\u00DA': // Ú [LATIN CAPITAL LETTER U WITH ACUTE] + case '\u00DB': // Û [LATIN CAPITAL LETTER U WITH CIRCUMFLEX] + case '\u00DC': // Ü [LATIN CAPITAL LETTER U WITH DIAERESIS] + case '\u0168': // Ũ [LATIN CAPITAL LETTER U WITH TILDE] + case '\u016A': // Ū [LATIN CAPITAL LETTER U WITH MACRON] + case '\u016C': // Ŭ [LATIN CAPITAL LETTER U WITH BREVE] + case '\u016E': // Ů [LATIN CAPITAL LETTER U WITH RING ABOVE] + case '\u0170': // Ű [LATIN CAPITAL LETTER U WITH DOUBLE ACUTE] + case '\u0172': // Ų [LATIN CAPITAL LETTER U WITH OGONEK] + case '\u01AF': // Ư [LATIN CAPITAL LETTER U WITH HORN] + case '\u01D3': // Ǔ [LATIN CAPITAL LETTER U WITH CARON] + case '\u01D5': // Ǖ [LATIN CAPITAL LETTER U WITH DIAERESIS AND MACRON] + case '\u01D7': // Ǘ [LATIN CAPITAL LETTER U WITH DIAERESIS AND ACUTE] + case '\u01D9': // Ǚ [LATIN CAPITAL LETTER U WITH DIAERESIS AND CARON] + case '\u01DB': // Ǜ [LATIN CAPITAL LETTER U WITH DIAERESIS AND GRAVE] + case '\u0214': // Ȕ [LATIN CAPITAL LETTER U WITH DOUBLE GRAVE] + case '\u0216': // Ȗ [LATIN CAPITAL LETTER U WITH INVERTED BREVE] + case '\u0244': // Ʉ [LATIN CAPITAL LETTER U BAR] + case '\u1D1C': // ᴜ [LATIN LETTER SMALL CAPITAL U] + case '\u1D7E': // ᵾ [LATIN SMALL CAPITAL LETTER U WITH STROKE] + case '\u1E72': // Ṳ [LATIN CAPITAL LETTER U WITH DIAERESIS BELOW] + case '\u1E74': // Ṵ [LATIN CAPITAL LETTER U WITH TILDE BELOW] + case '\u1E76': // Ṷ [LATIN CAPITAL LETTER U WITH CIRCUMFLEX BELOW] + case '\u1E78': // Ṹ [LATIN CAPITAL LETTER U WITH TILDE AND ACUTE] + case '\u1E7A': // Ṻ [LATIN CAPITAL LETTER U WITH MACRON AND DIAERESIS] + case '\u1EE4': // Ụ [LATIN CAPITAL LETTER U WITH DOT BELOW] + case '\u1EE6': // Ủ [LATIN CAPITAL LETTER U WITH HOOK ABOVE] + case '\u1EE8': // Ứ [LATIN CAPITAL LETTER U WITH HORN AND ACUTE] + case '\u1EEA': // Ừ [LATIN CAPITAL LETTER U WITH HORN AND GRAVE] + case '\u1EEC': // Ử [LATIN CAPITAL LETTER U WITH HORN AND HOOK ABOVE] + case '\u1EEE': // Ữ [LATIN CAPITAL LETTER U WITH HORN AND TILDE] + case '\u1EF0': // Ự [LATIN CAPITAL LETTER U WITH HORN AND DOT BELOW] + case '\u24CA': // Ⓤ [CIRCLED LATIN CAPITAL LETTER U] + case '\uFF35': // U [FULLWIDTH LATIN CAPITAL LETTER U] + output[outputPos++] = 'U'; + break; + case '\u00F9': // ù [LATIN SMALL LETTER U WITH GRAVE] + case '\u00FA': // ú [LATIN SMALL LETTER U WITH ACUTE] + case '\u00FB': // û [LATIN SMALL LETTER U WITH CIRCUMFLEX] + case '\u00FC': // ü [LATIN SMALL LETTER U WITH DIAERESIS] + case '\u0169': // ũ [LATIN SMALL LETTER U WITH TILDE] + case '\u016B': // ū [LATIN SMALL LETTER U WITH MACRON] + case '\u016D': // ŭ [LATIN SMALL LETTER U WITH BREVE] + case '\u016F': // ů [LATIN SMALL LETTER U WITH RING ABOVE] + case '\u0171': // ű [LATIN SMALL LETTER U WITH DOUBLE ACUTE] + case '\u0173': // ų [LATIN SMALL LETTER U WITH OGONEK] + case '\u01B0': // ư [LATIN SMALL LETTER U WITH HORN] + case '\u01D4': // ǔ [LATIN SMALL LETTER U WITH CARON] + case '\u01D6': // ǖ [LATIN SMALL LETTER U WITH DIAERESIS AND MACRON] + case '\u01D8': // ǘ [LATIN SMALL LETTER U WITH DIAERESIS AND ACUTE] + case '\u01DA': // ǚ [LATIN SMALL LETTER U WITH DIAERESIS AND CARON] + case '\u01DC': // ǜ [LATIN SMALL LETTER U WITH DIAERESIS AND GRAVE] + case '\u0215': // ȕ [LATIN SMALL LETTER U WITH DOUBLE GRAVE] + case '\u0217': // ȗ [LATIN SMALL LETTER U WITH INVERTED BREVE] + case '\u0289': // ʉ [LATIN SMALL LETTER U BAR] + case '\u1D64': // ᵤ [LATIN SUBSCRIPT SMALL LETTER U] + case '\u1D99': // ᶙ [LATIN SMALL LETTER U WITH RETROFLEX HOOK] + case '\u1E73': // ṳ [LATIN SMALL LETTER U WITH DIAERESIS BELOW] + case '\u1E75': // ṵ [LATIN SMALL LETTER U WITH TILDE BELOW] + case '\u1E77': // ṷ [LATIN SMALL LETTER U WITH CIRCUMFLEX BELOW] + case '\u1E79': // ṹ [LATIN SMALL LETTER U WITH TILDE AND ACUTE] + case '\u1E7B': // ṻ [LATIN SMALL LETTER U WITH MACRON AND DIAERESIS] + case '\u1EE5': // ụ [LATIN SMALL LETTER U WITH DOT BELOW] + case '\u1EE7': // ủ [LATIN SMALL LETTER U WITH HOOK ABOVE] + case '\u1EE9': // ứ [LATIN SMALL LETTER U WITH HORN AND ACUTE] + case '\u1EEB': // ừ [LATIN SMALL LETTER U WITH HORN AND GRAVE] + case '\u1EED': // ử [LATIN SMALL LETTER U WITH HORN AND HOOK ABOVE] + case '\u1EEF': // ữ [LATIN SMALL LETTER U WITH HORN AND TILDE] + case '\u1EF1': // ự [LATIN SMALL LETTER U WITH HORN AND DOT BELOW] + case '\u24E4': // ⓤ [CIRCLED LATIN SMALL LETTER U] + case '\uFF55': // u [FULLWIDTH LATIN SMALL LETTER U] + output[outputPos++] = 'u'; + break; + case '\u24B0': // ⒰ [PARENTHESIZED LATIN SMALL LETTER U] + output[outputPos++] = '('; + output[outputPos++] = 'u'; + output[outputPos++] = ')'; + break; + case '\u1D6B': // ᵫ [LATIN SMALL LETTER UE] + output[outputPos++] = 'u'; + output[outputPos++] = 'e'; + break; + case '\u01B2': // Ʋ [LATIN CAPITAL LETTER V WITH HOOK] + case '\u0245': // Ʌ [LATIN CAPITAL LETTER TURNED V] + case '\u1D20': // ᴠ [LATIN LETTER SMALL CAPITAL V] + case '\u1E7C': // Ṽ [LATIN CAPITAL LETTER V WITH TILDE] + case '\u1E7E': // Ṿ [LATIN CAPITAL LETTER V WITH DOT BELOW] + case '\u1EFC': // Ỽ [LATIN CAPITAL LETTER MIDDLE-WELSH V] + case '\u24CB': // Ⓥ [CIRCLED LATIN CAPITAL LETTER V] + case '\uA75E': // Ꝟ [LATIN CAPITAL LETTER V WITH DIAGONAL STROKE] + case '\uA768': // Ꝩ [LATIN CAPITAL LETTER VEND] + case '\uFF36': // V [FULLWIDTH LATIN CAPITAL LETTER V] + output[outputPos++] = 'V'; + break; + case '\u028B': // ʋ [LATIN SMALL LETTER V WITH HOOK] + case '\u028C': // ʌ [LATIN SMALL LETTER TURNED V] + case '\u1D65': // ᵥ [LATIN SUBSCRIPT SMALL LETTER V] + case '\u1D8C': // ᶌ [LATIN SMALL LETTER V WITH PALATAL HOOK] + case '\u1E7D': // ṽ [LATIN SMALL LETTER V WITH TILDE] + case '\u1E7F': // ṿ [LATIN SMALL LETTER V WITH DOT BELOW] + case '\u24E5': // ⓥ [CIRCLED LATIN SMALL LETTER V] + case '\u2C71': // ⱱ [LATIN SMALL LETTER V WITH RIGHT HOOK] + case '\u2C74': // ⱴ [LATIN SMALL LETTER V WITH CURL] + case '\uA75F': // ꝟ [LATIN SMALL LETTER V WITH DIAGONAL STROKE] + case '\uFF56': // v [FULLWIDTH LATIN SMALL LETTER V] + output[outputPos++] = 'v'; + break; + case '\uA760': // Ꝡ [LATIN CAPITAL LETTER VY] + output[outputPos++] = 'V'; + output[outputPos++] = 'Y'; + break; + case '\u24B1': // ⒱ [PARENTHESIZED LATIN SMALL LETTER V] + output[outputPos++] = '('; + output[outputPos++] = 'v'; + output[outputPos++] = ')'; + break; + case '\uA761': // ꝡ [LATIN SMALL LETTER VY] + output[outputPos++] = 'v'; + output[outputPos++] = 'y'; + break; + case '\u0174': // Ŵ [LATIN CAPITAL LETTER W WITH CIRCUMFLEX] + case '\u01F7': // Ƿ http://en.wikipedia.org/wiki/Wynn [LATIN CAPITAL LETTER WYNN] + case '\u1D21': // ᴡ [LATIN LETTER SMALL CAPITAL W] + case '\u1E80': // Ẁ [LATIN CAPITAL LETTER W WITH GRAVE] + case '\u1E82': // Ẃ [LATIN CAPITAL LETTER W WITH ACUTE] + case '\u1E84': // Ẅ [LATIN CAPITAL LETTER W WITH DIAERESIS] + case '\u1E86': // Ẇ [LATIN CAPITAL LETTER W WITH DOT ABOVE] + case '\u1E88': // Ẉ [LATIN CAPITAL LETTER W WITH DOT BELOW] + case '\u24CC': // Ⓦ [CIRCLED LATIN CAPITAL LETTER W] + case '\u2C72': // Ⱳ [LATIN CAPITAL LETTER W WITH HOOK] + case '\uFF37': // W [FULLWIDTH LATIN CAPITAL LETTER W] + output[outputPos++] = 'W'; + break; + case '\u0175': // ŵ [LATIN SMALL LETTER W WITH CIRCUMFLEX] + case '\u01BF': // ƿ http://en.wikipedia.org/wiki/Wynn [LATIN LETTER WYNN] + case '\u028D': // ʍ [LATIN SMALL LETTER TURNED W] + case '\u1E81': // ẁ [LATIN SMALL LETTER W WITH GRAVE] + case '\u1E83': // ẃ [LATIN SMALL LETTER W WITH ACUTE] + case '\u1E85': // ẅ [LATIN SMALL LETTER W WITH DIAERESIS] + case '\u1E87': // ẇ [LATIN SMALL LETTER W WITH DOT ABOVE] + case '\u1E89': // ẉ [LATIN SMALL LETTER W WITH DOT BELOW] + case '\u1E98': // ẘ [LATIN SMALL LETTER W WITH RING ABOVE] + case '\u24E6': // ⓦ [CIRCLED LATIN SMALL LETTER W] + case '\u2C73': // ⱳ [LATIN SMALL LETTER W WITH HOOK] + case '\uFF57': // w [FULLWIDTH LATIN SMALL LETTER W] + output[outputPos++] = 'w'; + break; + case '\u24B2': // ⒲ [PARENTHESIZED LATIN SMALL LETTER W] + output[outputPos++] = '('; + output[outputPos++] = 'w'; + output[outputPos++] = ')'; + break; + case '\u1E8A': // Ẋ [LATIN CAPITAL LETTER X WITH DOT ABOVE] + case '\u1E8C': // Ẍ [LATIN CAPITAL LETTER X WITH DIAERESIS] + case '\u24CD': // Ⓧ [CIRCLED LATIN CAPITAL LETTER X] + case '\uFF38': // X [FULLWIDTH LATIN CAPITAL LETTER X] + output[outputPos++] = 'X'; + break; + case '\u1D8D': // ᶍ [LATIN SMALL LETTER X WITH PALATAL HOOK] + case '\u1E8B': // ẋ [LATIN SMALL LETTER X WITH DOT ABOVE] + case '\u1E8D': // ẍ [LATIN SMALL LETTER X WITH DIAERESIS] + case '\u2093': // ₓ [LATIN SUBSCRIPT SMALL LETTER X] + case '\u24E7': // ⓧ [CIRCLED LATIN SMALL LETTER X] + case '\uFF58': // x [FULLWIDTH LATIN SMALL LETTER X] + output[outputPos++] = 'x'; + break; + case '\u24B3': // ⒳ [PARENTHESIZED LATIN SMALL LETTER X] + output[outputPos++] = '('; + output[outputPos++] = 'x'; + output[outputPos++] = ')'; + break; + case '\u00DD': // Ý [LATIN CAPITAL LETTER Y WITH ACUTE] + case '\u0176': // Ŷ [LATIN CAPITAL LETTER Y WITH CIRCUMFLEX] + case '\u0178': // Ÿ [LATIN CAPITAL LETTER Y WITH DIAERESIS] + case '\u01B3': // Ƴ [LATIN CAPITAL LETTER Y WITH HOOK] + case '\u0232': // Ȳ [LATIN CAPITAL LETTER Y WITH MACRON] + case '\u024E': // Ɏ [LATIN CAPITAL LETTER Y WITH STROKE] + case '\u028F': // ʏ [LATIN LETTER SMALL CAPITAL Y] + case '\u1E8E': // Ẏ [LATIN CAPITAL LETTER Y WITH DOT ABOVE] + case '\u1EF2': // Ỳ [LATIN CAPITAL LETTER Y WITH GRAVE] + case '\u1EF4': // Ỵ [LATIN CAPITAL LETTER Y WITH DOT BELOW] + case '\u1EF6': // Ỷ [LATIN CAPITAL LETTER Y WITH HOOK ABOVE] + case '\u1EF8': // Ỹ [LATIN CAPITAL LETTER Y WITH TILDE] + case '\u1EFE': // Ỿ [LATIN CAPITAL LETTER Y WITH LOOP] + case '\u24CE': // Ⓨ [CIRCLED LATIN CAPITAL LETTER Y] + case '\uFF39': // Y [FULLWIDTH LATIN CAPITAL LETTER Y] + output[outputPos++] = 'Y'; + break; + case '\u00FD': // ý [LATIN SMALL LETTER Y WITH ACUTE] + case '\u00FF': // ÿ [LATIN SMALL LETTER Y WITH DIAERESIS] + case '\u0177': // ŷ [LATIN SMALL LETTER Y WITH CIRCUMFLEX] + case '\u01B4': // ƴ [LATIN SMALL LETTER Y WITH HOOK] + case '\u0233': // ȳ [LATIN SMALL LETTER Y WITH MACRON] + case '\u024F': // ɏ [LATIN SMALL LETTER Y WITH STROKE] + case '\u028E': // ʎ [LATIN SMALL LETTER TURNED Y] + case '\u1E8F': // ẏ [LATIN SMALL LETTER Y WITH DOT ABOVE] + case '\u1E99': // ẙ [LATIN SMALL LETTER Y WITH RING ABOVE] + case '\u1EF3': // ỳ [LATIN SMALL LETTER Y WITH GRAVE] + case '\u1EF5': // ỵ [LATIN SMALL LETTER Y WITH DOT BELOW] + case '\u1EF7': // ỷ [LATIN SMALL LETTER Y WITH HOOK ABOVE] + case '\u1EF9': // ỹ [LATIN SMALL LETTER Y WITH TILDE] + case '\u1EFF': // ỿ [LATIN SMALL LETTER Y WITH LOOP] + case '\u24E8': // ⓨ [CIRCLED LATIN SMALL LETTER Y] + case '\uFF59': // y [FULLWIDTH LATIN SMALL LETTER Y] + output[outputPos++] = 'y'; + break; + case '\u24B4': // ⒴ [PARENTHESIZED LATIN SMALL LETTER Y] + output[outputPos++] = '('; + output[outputPos++] = 'y'; + output[outputPos++] = ')'; + break; + case '\u0179': // Ź [LATIN CAPITAL LETTER Z WITH ACUTE] + case '\u017B': // Ż [LATIN CAPITAL LETTER Z WITH DOT ABOVE] + case '\u017D': // Ž [LATIN CAPITAL LETTER Z WITH CARON] + case '\u01B5': // Ƶ [LATIN CAPITAL LETTER Z WITH STROKE] + case '\u021C': // Ȝ http://en.wikipedia.org/wiki/Yogh [LATIN CAPITAL LETTER YOGH] + case '\u0224': // Ȥ [LATIN CAPITAL LETTER Z WITH HOOK] + case '\u1D22': // ᴢ [LATIN LETTER SMALL CAPITAL Z] + case '\u1E90': // Ẑ [LATIN CAPITAL LETTER Z WITH CIRCUMFLEX] + case '\u1E92': // Ẓ [LATIN CAPITAL LETTER Z WITH DOT BELOW] + case '\u1E94': // Ẕ [LATIN CAPITAL LETTER Z WITH LINE BELOW] + case '\u24CF': // Ⓩ [CIRCLED LATIN CAPITAL LETTER Z] + case '\u2C6B': // Ⱬ [LATIN CAPITAL LETTER Z WITH DESCENDER] + case '\uA762': // Ꝣ [LATIN CAPITAL LETTER VISIGOTHIC Z] + case '\uFF3A': // Z [FULLWIDTH LATIN CAPITAL LETTER Z] + output[outputPos++] = 'Z'; + break; + case '\u017A': // ź [LATIN SMALL LETTER Z WITH ACUTE] + case '\u017C': // ż [LATIN SMALL LETTER Z WITH DOT ABOVE] + case '\u017E': // ž [LATIN SMALL LETTER Z WITH CARON] + case '\u01B6': // ƶ [LATIN SMALL LETTER Z WITH STROKE] + case '\u021D': // ȝ http://en.wikipedia.org/wiki/Yogh [LATIN SMALL LETTER YOGH] + case '\u0225': // ȥ [LATIN SMALL LETTER Z WITH HOOK] + case '\u0240': // ɀ [LATIN SMALL LETTER Z WITH SWASH TAIL] + case '\u0290': // ʐ [LATIN SMALL LETTER Z WITH RETROFLEX HOOK] + case '\u0291': // ʑ [LATIN SMALL LETTER Z WITH CURL] + case '\u1D76': // ᵶ [LATIN SMALL LETTER Z WITH MIDDLE TILDE] + case '\u1D8E': // ᶎ [LATIN SMALL LETTER Z WITH PALATAL HOOK] + case '\u1E91': // ẑ [LATIN SMALL LETTER Z WITH CIRCUMFLEX] + case '\u1E93': // ẓ [LATIN SMALL LETTER Z WITH DOT BELOW] + case '\u1E95': // ẕ [LATIN SMALL LETTER Z WITH LINE BELOW] + case '\u24E9': // ⓩ [CIRCLED LATIN SMALL LETTER Z] + case '\u2C6C': // ⱬ [LATIN SMALL LETTER Z WITH DESCENDER] + case '\uA763': // ꝣ [LATIN SMALL LETTER VISIGOTHIC Z] + case '\uFF5A': // z [FULLWIDTH LATIN SMALL LETTER Z] + output[outputPos++] = 'z'; + break; + case '\u24B5': // ⒵ [PARENTHESIZED LATIN SMALL LETTER Z] + output[outputPos++] = '('; + output[outputPos++] = 'z'; + output[outputPos++] = ')'; + break; + case '\u2070': // ⁰ [SUPERSCRIPT ZERO] + case '\u2080': // ₀ [SUBSCRIPT ZERO] + case '\u24EA': // ⓪ [CIRCLED DIGIT ZERO] + case '\u24FF': // ⓿ [NEGATIVE CIRCLED DIGIT ZERO] + case '\uFF10': // 0 [FULLWIDTH DIGIT ZERO] + output[outputPos++] = '0'; + break; + case '\u00B9': // ¹ [SUPERSCRIPT ONE] + case '\u2081': // ₁ [SUBSCRIPT ONE] + case '\u2460': // ① [CIRCLED DIGIT ONE] + case '\u24F5': // ⓵ [DOUBLE CIRCLED DIGIT ONE] + case '\u2776': // ❶ [DINGBAT NEGATIVE CIRCLED DIGIT ONE] + case '\u2780': // ➀ [DINGBAT CIRCLED SANS-SERIF DIGIT ONE] + case '\u278A': // ➊ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT ONE] + case '\uFF11': // 1 [FULLWIDTH DIGIT ONE] + output[outputPos++] = '1'; + break; + case '\u2488': // ⒈ [DIGIT ONE FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '.'; + break; + case '\u2474': // ⑴ [PARENTHESIZED DIGIT ONE] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = ')'; + break; + case '\u00B2': // ² [SUPERSCRIPT TWO] + case '\u2082': // ₂ [SUBSCRIPT TWO] + case '\u2461': // ② [CIRCLED DIGIT TWO] + case '\u24F6': // ⓶ [DOUBLE CIRCLED DIGIT TWO] + case '\u2777': // ❷ [DINGBAT NEGATIVE CIRCLED DIGIT TWO] + case '\u2781': // ➁ [DINGBAT CIRCLED SANS-SERIF DIGIT TWO] + case '\u278B': // ➋ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT TWO] + case '\uFF12': // 2 [FULLWIDTH DIGIT TWO] + output[outputPos++] = '2'; + break; + case '\u2489': // ⒉ [DIGIT TWO FULL STOP] + output[outputPos++] = '2'; + output[outputPos++] = '.'; + break; + case '\u2475': // ⑵ [PARENTHESIZED DIGIT TWO] + output[outputPos++] = '('; + output[outputPos++] = '2'; + output[outputPos++] = ')'; + break; + case '\u00B3': // ³ [SUPERSCRIPT THREE] + case '\u2083': // ₃ [SUBSCRIPT THREE] + case '\u2462': // ③ [CIRCLED DIGIT THREE] + case '\u24F7': // ⓷ [DOUBLE CIRCLED DIGIT THREE] + case '\u2778': // ❸ [DINGBAT NEGATIVE CIRCLED DIGIT THREE] + case '\u2782': // ➂ [DINGBAT CIRCLED SANS-SERIF DIGIT THREE] + case '\u278C': // ➌ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT THREE] + case '\uFF13': // 3 [FULLWIDTH DIGIT THREE] + output[outputPos++] = '3'; + break; + case '\u248A': // ⒊ [DIGIT THREE FULL STOP] + output[outputPos++] = '3'; + output[outputPos++] = '.'; + break; + case '\u2476': // ⑶ [PARENTHESIZED DIGIT THREE] + output[outputPos++] = '('; + output[outputPos++] = '3'; + output[outputPos++] = ')'; + break; + case '\u2074': // ⁴ [SUPERSCRIPT FOUR] + case '\u2084': // ₄ [SUBSCRIPT FOUR] + case '\u2463': // ④ [CIRCLED DIGIT FOUR] + case '\u24F8': // ⓸ [DOUBLE CIRCLED DIGIT FOUR] + case '\u2779': // ❹ [DINGBAT NEGATIVE CIRCLED DIGIT FOUR] + case '\u2783': // ➃ [DINGBAT CIRCLED SANS-SERIF DIGIT FOUR] + case '\u278D': // ➍ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT FOUR] + case '\uFF14': // 4 [FULLWIDTH DIGIT FOUR] + output[outputPos++] = '4'; + break; + case '\u248B': // ⒋ [DIGIT FOUR FULL STOP] + output[outputPos++] = '4'; + output[outputPos++] = '.'; + break; + case '\u2477': // ⑷ [PARENTHESIZED DIGIT FOUR] + output[outputPos++] = '('; + output[outputPos++] = '4'; + output[outputPos++] = ')'; + break; + case '\u2075': // ⁵ [SUPERSCRIPT FIVE] + case '\u2085': // ₅ [SUBSCRIPT FIVE] + case '\u2464': // ⑤ [CIRCLED DIGIT FIVE] + case '\u24F9': // ⓹ [DOUBLE CIRCLED DIGIT FIVE] + case '\u277A': // ❺ [DINGBAT NEGATIVE CIRCLED DIGIT FIVE] + case '\u2784': // ➄ [DINGBAT CIRCLED SANS-SERIF DIGIT FIVE] + case '\u278E': // ➎ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT FIVE] + case '\uFF15': // 5 [FULLWIDTH DIGIT FIVE] + output[outputPos++] = '5'; + break; + case '\u248C': // ⒌ [DIGIT FIVE FULL STOP] + output[outputPos++] = '5'; + output[outputPos++] = '.'; + break; + case '\u2478': // ⑸ [PARENTHESIZED DIGIT FIVE] + output[outputPos++] = '('; + output[outputPos++] = '5'; + output[outputPos++] = ')'; + break; + case '\u2076': // ⁶ [SUPERSCRIPT SIX] + case '\u2086': // ₆ [SUBSCRIPT SIX] + case '\u2465': // ⑥ [CIRCLED DIGIT SIX] + case '\u24FA': // ⓺ [DOUBLE CIRCLED DIGIT SIX] + case '\u277B': // ❻ [DINGBAT NEGATIVE CIRCLED DIGIT SIX] + case '\u2785': // ➅ [DINGBAT CIRCLED SANS-SERIF DIGIT SIX] + case '\u278F': // ➏ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT SIX] + case '\uFF16': // 6 [FULLWIDTH DIGIT SIX] + output[outputPos++] = '6'; + break; + case '\u248D': // ⒍ [DIGIT SIX FULL STOP] + output[outputPos++] = '6'; + output[outputPos++] = '.'; + break; + case '\u2479': // ⑹ [PARENTHESIZED DIGIT SIX] + output[outputPos++] = '('; + output[outputPos++] = '6'; + output[outputPos++] = ')'; + break; + case '\u2077': // ⁷ [SUPERSCRIPT SEVEN] + case '\u2087': // ₇ [SUBSCRIPT SEVEN] + case '\u2466': // ⑦ [CIRCLED DIGIT SEVEN] + case '\u24FB': // ⓻ [DOUBLE CIRCLED DIGIT SEVEN] + case '\u277C': // ❼ [DINGBAT NEGATIVE CIRCLED DIGIT SEVEN] + case '\u2786': // ➆ [DINGBAT CIRCLED SANS-SERIF DIGIT SEVEN] + case '\u2790': // ➐ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT SEVEN] + case '\uFF17': // 7 [FULLWIDTH DIGIT SEVEN] + output[outputPos++] = '7'; + break; + case '\u248E': // ⒎ [DIGIT SEVEN FULL STOP] + output[outputPos++] = '7'; + output[outputPos++] = '.'; + break; + case '\u247A': // ⑺ [PARENTHESIZED DIGIT SEVEN] + output[outputPos++] = '('; + output[outputPos++] = '7'; + output[outputPos++] = ')'; + break; + case '\u2078': // ⁸ [SUPERSCRIPT EIGHT] + case '\u2088': // ₈ [SUBSCRIPT EIGHT] + case '\u2467': // ⑧ [CIRCLED DIGIT EIGHT] + case '\u24FC': // ⓼ [DOUBLE CIRCLED DIGIT EIGHT] + case '\u277D': // ❽ [DINGBAT NEGATIVE CIRCLED DIGIT EIGHT] + case '\u2787': // ➇ [DINGBAT CIRCLED SANS-SERIF DIGIT EIGHT] + case '\u2791': // ➑ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT EIGHT] + case '\uFF18': // 8 [FULLWIDTH DIGIT EIGHT] + output[outputPos++] = '8'; + break; + case '\u248F': // ⒏ [DIGIT EIGHT FULL STOP] + output[outputPos++] = '8'; + output[outputPos++] = '.'; + break; + case '\u247B': // ⑻ [PARENTHESIZED DIGIT EIGHT] + output[outputPos++] = '('; + output[outputPos++] = '8'; + output[outputPos++] = ')'; + break; + case '\u2079': // ⁹ [SUPERSCRIPT NINE] + case '\u2089': // ₉ [SUBSCRIPT NINE] + case '\u2468': // ⑨ [CIRCLED DIGIT NINE] + case '\u24FD': // ⓽ [DOUBLE CIRCLED DIGIT NINE] + case '\u277E': // ❾ [DINGBAT NEGATIVE CIRCLED DIGIT NINE] + case '\u2788': // ➈ [DINGBAT CIRCLED SANS-SERIF DIGIT NINE] + case '\u2792': // ➒ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT NINE] + case '\uFF19': // 9 [FULLWIDTH DIGIT NINE] + output[outputPos++] = '9'; + break; + case '\u2490': // ⒐ [DIGIT NINE FULL STOP] + output[outputPos++] = '9'; + output[outputPos++] = '.'; + break; + case '\u247C': // ⑼ [PARENTHESIZED DIGIT NINE] + output[outputPos++] = '('; + output[outputPos++] = '9'; + output[outputPos++] = ')'; + break; + case '\u2469': // ⑩ [CIRCLED NUMBER TEN] + case '\u24FE': // ⓾ [DOUBLE CIRCLED NUMBER TEN] + case '\u277F': // ❿ [DINGBAT NEGATIVE CIRCLED NUMBER TEN] + case '\u2789': // ➉ [DINGBAT CIRCLED SANS-SERIF NUMBER TEN] + case '\u2793': // ➓ [DINGBAT NEGATIVE CIRCLED SANS-SERIF NUMBER TEN] + output[outputPos++] = '1'; + output[outputPos++] = '0'; + break; + case '\u2491': // ⒑ [NUMBER TEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '0'; + output[outputPos++] = '.'; + break; + case '\u247D': // ⑽ [PARENTHESIZED NUMBER TEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '0'; + output[outputPos++] = ')'; + break; + case '\u246A': // ⑪ [CIRCLED NUMBER ELEVEN] + case '\u24EB': // ⓫ [NEGATIVE CIRCLED NUMBER ELEVEN] + output[outputPos++] = '1'; + output[outputPos++] = '1'; + break; + case '\u2492': // ⒒ [NUMBER ELEVEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '1'; + output[outputPos++] = '.'; + break; + case '\u247E': // ⑾ [PARENTHESIZED NUMBER ELEVEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '1'; + output[outputPos++] = ')'; + break; + case '\u246B': // ⑫ [CIRCLED NUMBER TWELVE] + case '\u24EC': // ⓬ [NEGATIVE CIRCLED NUMBER TWELVE] + output[outputPos++] = '1'; + output[outputPos++] = '2'; + break; + case '\u2493': // ⒓ [NUMBER TWELVE FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '2'; + output[outputPos++] = '.'; + break; + case '\u247F': // ⑿ [PARENTHESIZED NUMBER TWELVE] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '2'; + output[outputPos++] = ')'; + break; + case '\u246C': // ⑬ [CIRCLED NUMBER THIRTEEN] + case '\u24ED': // ⓭ [NEGATIVE CIRCLED NUMBER THIRTEEN] + output[outputPos++] = '1'; + output[outputPos++] = '3'; + break; + case '\u2494': // ⒔ [NUMBER THIRTEEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '3'; + output[outputPos++] = '.'; + break; + case '\u2480': // ⒀ [PARENTHESIZED NUMBER THIRTEEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '3'; + output[outputPos++] = ')'; + break; + case '\u246D': // ⑭ [CIRCLED NUMBER FOURTEEN] + case '\u24EE': // ⓮ [NEGATIVE CIRCLED NUMBER FOURTEEN] + output[outputPos++] = '1'; + output[outputPos++] = '4'; + break; + case '\u2495': // ⒕ [NUMBER FOURTEEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '4'; + output[outputPos++] = '.'; + break; + case '\u2481': // ⒁ [PARENTHESIZED NUMBER FOURTEEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '4'; + output[outputPos++] = ')'; + break; + case '\u246E': // ⑮ [CIRCLED NUMBER FIFTEEN] + case '\u24EF': // ⓯ [NEGATIVE CIRCLED NUMBER FIFTEEN] + output[outputPos++] = '1'; + output[outputPos++] = '5'; + break; + case '\u2496': // ⒖ [NUMBER FIFTEEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '5'; + output[outputPos++] = '.'; + break; + case '\u2482': // ⒂ [PARENTHESIZED NUMBER FIFTEEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '5'; + output[outputPos++] = ')'; + break; + case '\u246F': // ⑯ [CIRCLED NUMBER SIXTEEN] + case '\u24F0': // ⓰ [NEGATIVE CIRCLED NUMBER SIXTEEN] + output[outputPos++] = '1'; + output[outputPos++] = '6'; + break; + case '\u2497': // ⒗ [NUMBER SIXTEEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '6'; + output[outputPos++] = '.'; + break; + case '\u2483': // ⒃ [PARENTHESIZED NUMBER SIXTEEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '6'; + output[outputPos++] = ')'; + break; + case '\u2470': // ⑰ [CIRCLED NUMBER SEVENTEEN] + case '\u24F1': // ⓱ [NEGATIVE CIRCLED NUMBER SEVENTEEN] + output[outputPos++] = '1'; + output[outputPos++] = '7'; + break; + case '\u2498': // ⒘ [NUMBER SEVENTEEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '7'; + output[outputPos++] = '.'; + break; + case '\u2484': // ⒄ [PARENTHESIZED NUMBER SEVENTEEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '7'; + output[outputPos++] = ')'; + break; + case '\u2471': // ⑱ [CIRCLED NUMBER EIGHTEEN] + case '\u24F2': // ⓲ [NEGATIVE CIRCLED NUMBER EIGHTEEN] + output[outputPos++] = '1'; + output[outputPos++] = '8'; + break; + case '\u2499': // ⒙ [NUMBER EIGHTEEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '8'; + output[outputPos++] = '.'; + break; + case '\u2485': // ⒅ [PARENTHESIZED NUMBER EIGHTEEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '8'; + output[outputPos++] = ')'; + break; + case '\u2472': // ⑲ [CIRCLED NUMBER NINETEEN] + case '\u24F3': // ⓳ [NEGATIVE CIRCLED NUMBER NINETEEN] + output[outputPos++] = '1'; + output[outputPos++] = '9'; + break; + case '\u249A': // ⒚ [NUMBER NINETEEN FULL STOP] + output[outputPos++] = '1'; + output[outputPos++] = '9'; + output[outputPos++] = '.'; + break; + case '\u2486': // ⒆ [PARENTHESIZED NUMBER NINETEEN] + output[outputPos++] = '('; + output[outputPos++] = '1'; + output[outputPos++] = '9'; + output[outputPos++] = ')'; + break; + case '\u2473': // ⑳ [CIRCLED NUMBER TWENTY] + case '\u24F4': // ⓴ [NEGATIVE CIRCLED NUMBER TWENTY] + output[outputPos++] = '2'; + output[outputPos++] = '0'; + break; + case '\u249B': // ⒛ [NUMBER TWENTY FULL STOP] + output[outputPos++] = '2'; + output[outputPos++] = '0'; + output[outputPos++] = '.'; + break; + case '\u2487': // ⒇ [PARENTHESIZED NUMBER TWENTY] + output[outputPos++] = '('; + output[outputPos++] = '2'; + output[outputPos++] = '0'; + output[outputPos++] = ')'; + break; + case '\u00AB': // « [LEFT-POINTING DOUBLE ANGLE QUOTATION MARK] + case '\u00BB': // » [RIGHT-POINTING DOUBLE ANGLE QUOTATION MARK] + case '\u201C': // “ [LEFT DOUBLE QUOTATION MARK] + case '\u201D': // ” [RIGHT DOUBLE QUOTATION MARK] + case '\u201E': // „ [DOUBLE LOW-9 QUOTATION MARK] + case '\u2033': // ″ [DOUBLE PRIME] + case '\u2036': // ‶ [REVERSED DOUBLE PRIME] + case '\u275D': // ❝ [HEAVY DOUBLE TURNED COMMA QUOTATION MARK ORNAMENT] + case '\u275E': // ❞ [HEAVY DOUBLE COMMA QUOTATION MARK ORNAMENT] + case '\u276E': // ❮ [HEAVY LEFT-POINTING ANGLE QUOTATION MARK ORNAMENT] + case '\u276F': // ❯ [HEAVY RIGHT-POINTING ANGLE QUOTATION MARK ORNAMENT] + case '\uFF02': // " [FULLWIDTH QUOTATION MARK] + output[outputPos++] = '"'; + break; + case '\u2018': // ‘ [LEFT SINGLE QUOTATION MARK] + case '\u2019': // ’ [RIGHT SINGLE QUOTATION MARK] + case '\u201A': // ‚ [SINGLE LOW-9 QUOTATION MARK] + case '\u201B': // ‛ [SINGLE HIGH-REVERSED-9 QUOTATION MARK] + case '\u2032': // ′ [PRIME] + case '\u2035': // ‵ [REVERSED PRIME] + case '\u2039': // ‹ [SINGLE LEFT-POINTING ANGLE QUOTATION MARK] + case '\u203A': // › [SINGLE RIGHT-POINTING ANGLE QUOTATION MARK] + case '\u275B': // ❛ [HEAVY SINGLE TURNED COMMA QUOTATION MARK ORNAMENT] + case '\u275C': // ❜ [HEAVY SINGLE COMMA QUOTATION MARK ORNAMENT] + case '\uFF07': // ' [FULLWIDTH APOSTROPHE] + output[outputPos++] = '\''; + break; + case '\u2010': // ‐ [HYPHEN] + case '\u2011': // ‑ [NON-BREAKING HYPHEN] + case '\u2012': // ‒ [FIGURE DASH] + case '\u2013': // – [EN DASH] + case '\u2014': // — [EM DASH] + case '\u207B': // ⁻ [SUPERSCRIPT MINUS] + case '\u208B': // ₋ [SUBSCRIPT MINUS] + case '\uFF0D': // - [FULLWIDTH HYPHEN-MINUS] + output[outputPos++] = '-'; + break; + case '\u2045': // ⁅ [LEFT SQUARE BRACKET WITH QUILL] + case '\u2772': // ❲ [LIGHT LEFT TORTOISE SHELL BRACKET ORNAMENT] + case '\uFF3B': // [ [FULLWIDTH LEFT SQUARE BRACKET] + output[outputPos++] = '['; + break; + case '\u2046': // ⁆ [RIGHT SQUARE BRACKET WITH QUILL] + case '\u2773': // ❳ [LIGHT RIGHT TORTOISE SHELL BRACKET ORNAMENT] + case '\uFF3D': // ] [FULLWIDTH RIGHT SQUARE BRACKET] + output[outputPos++] = ']'; + break; + case '\u207D': // ⁽ [SUPERSCRIPT LEFT PARENTHESIS] + case '\u208D': // ₍ [SUBSCRIPT LEFT PARENTHESIS] + case '\u2768': // ❨ [MEDIUM LEFT PARENTHESIS ORNAMENT] + case '\u276A': // ❪ [MEDIUM FLATTENED LEFT PARENTHESIS ORNAMENT] + case '\uFF08': // ( [FULLWIDTH LEFT PARENTHESIS] + output[outputPos++] = '('; + break; + case '\u2E28': // ⸨ [LEFT DOUBLE PARENTHESIS] + output[outputPos++] = '('; + output[outputPos++] = '('; + break; + case '\u207E': // ⁾ [SUPERSCRIPT RIGHT PARENTHESIS] + case '\u208E': // ₎ [SUBSCRIPT RIGHT PARENTHESIS] + case '\u2769': // ❩ [MEDIUM RIGHT PARENTHESIS ORNAMENT] + case '\u276B': // ❫ [MEDIUM FLATTENED RIGHT PARENTHESIS ORNAMENT] + case '\uFF09': // ) [FULLWIDTH RIGHT PARENTHESIS] + output[outputPos++] = ')'; + break; + case '\u2E29': // ⸩ [RIGHT DOUBLE PARENTHESIS] + output[outputPos++] = ')'; + output[outputPos++] = ')'; + break; + case '\u276C': // ❬ [MEDIUM LEFT-POINTING ANGLE BRACKET ORNAMENT] + case '\u2770': // ❰ [HEAVY LEFT-POINTING ANGLE BRACKET ORNAMENT] + case '\uFF1C': // < [FULLWIDTH LESS-THAN SIGN] + output[outputPos++] = '<'; + break; + case '\u276D': // ❭ [MEDIUM RIGHT-POINTING ANGLE BRACKET ORNAMENT] + case '\u2771': // ❱ [HEAVY RIGHT-POINTING ANGLE BRACKET ORNAMENT] + case '\uFF1E': // > [FULLWIDTH GREATER-THAN SIGN] + output[outputPos++] = '>'; + break; + case '\u2774': // ❴ [MEDIUM LEFT CURLY BRACKET ORNAMENT] + case '\uFF5B': // { [FULLWIDTH LEFT CURLY BRACKET] + output[outputPos++] = '{'; + break; + case '\u2775': // ❵ [MEDIUM RIGHT CURLY BRACKET ORNAMENT] + case '\uFF5D': // } [FULLWIDTH RIGHT CURLY BRACKET] + output[outputPos++] = '}'; + break; + case '\u207A': // ⁺ [SUPERSCRIPT PLUS SIGN] + case '\u208A': // ₊ [SUBSCRIPT PLUS SIGN] + case '\uFF0B': // + [FULLWIDTH PLUS SIGN] + output[outputPos++] = '+'; + break; + case '\u207C': // ⁼ [SUPERSCRIPT EQUALS SIGN] + case '\u208C': // ₌ [SUBSCRIPT EQUALS SIGN] + case '\uFF1D': // = [FULLWIDTH EQUALS SIGN] + output[outputPos++] = '='; + break; + case '\uFF01': // ! [FULLWIDTH EXCLAMATION MARK] + output[outputPos++] = '!'; + break; + case '\u203C': // ‼ [DOUBLE EXCLAMATION MARK] + output[outputPos++] = '!'; + output[outputPos++] = '!'; + break; + case '\u2049': // ⁉ [EXCLAMATION QUESTION MARK] + output[outputPos++] = '!'; + output[outputPos++] = '?'; + break; + case '\uFF03': // # [FULLWIDTH NUMBER SIGN] + output[outputPos++] = '#'; + break; + case '\uFF04': // $ [FULLWIDTH DOLLAR SIGN] + output[outputPos++] = '$'; + break; + case '\u2052': // ⁒ [COMMERCIAL MINUS SIGN] + case '\uFF05': // % [FULLWIDTH PERCENT SIGN] + output[outputPos++] = '%'; + break; + case '\uFF06': // & [FULLWIDTH AMPERSAND] + output[outputPos++] = '&'; + break; + case '\u204E': // ⁎ [LOW ASTERISK] + case '\uFF0A': // * [FULLWIDTH ASTERISK] + output[outputPos++] = '*'; + break; + case '\uFF0C': // , [FULLWIDTH COMMA] + output[outputPos++] = ','; + break; + case '\uFF0E': // . [FULLWIDTH FULL STOP] + output[outputPos++] = '.'; + break; + case '\u2044': // ⁄ [FRACTION SLASH] + case '\uFF0F': // / [FULLWIDTH SOLIDUS] + output[outputPos++] = '/'; + break; + case '\uFF1A': // : [FULLWIDTH COLON] + output[outputPos++] = ':'; + break; + case '\u204F': // ⁏ [REVERSED SEMICOLON] + case '\uFF1B': // ; [FULLWIDTH SEMICOLON] + output[outputPos++] = ';'; + break; + case '\uFF1F': // ? [FULLWIDTH QUESTION MARK] + output[outputPos++] = '?'; + break; + case '\u2047': // ⁇ [DOUBLE QUESTION MARK] + output[outputPos++] = '?'; + output[outputPos++] = '?'; + break; + case '\u2048': // ⁈ [QUESTION EXCLAMATION MARK] + output[outputPos++] = '?'; + output[outputPos++] = '!'; + break; + case '\uFF20': // @ [FULLWIDTH COMMERCIAL AT] + output[outputPos++] = '@'; + break; + case '\uFF3C': // \ [FULLWIDTH REVERSE SOLIDUS] + output[outputPos++] = '\\'; + break; + case '\u2038': // ‸ [CARET] + case '\uFF3E': // ^ [FULLWIDTH CIRCUMFLEX ACCENT] + output[outputPos++] = '^'; + break; + case '\uFF3F': // _ [FULLWIDTH LOW LINE] + output[outputPos++] = '_'; + break; + case '\u2053': // ⁓ [SWUNG DASH] + case '\uFF5E': // ~ [FULLWIDTH TILDE] + output[outputPos++] = '~'; + break; + default: + output[outputPos++] = c; + break; + } + } + } + return outputPos; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineBuilder.java b/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineBuilder.java new file mode 100644 index 000000000000..3a6a72603df2 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineBuilder.java @@ -0,0 +1,52 @@ +/* + * 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.index.sai.analyzer.filter; + +/** + * Creates a Pipeline object for applying n pieces of logic + * from the provided methods to the builder in a guaranteed order + */ +public class FilterPipelineBuilder +{ + private final FilterPipelineTask parent; + private FilterPipelineTask current; + + public FilterPipelineBuilder(FilterPipelineTask first) + { + this(first, first); + } + + private FilterPipelineBuilder(FilterPipelineTask first, FilterPipelineTask current) + { + this.parent = first; + this.current = current; + } + + public FilterPipelineBuilder add(String name, FilterPipelineTask nextTask) + { + this.current.setLast(name, nextTask); + this.current = nextTask; + return this; + } + + public FilterPipelineTask build() + { + return this.parent; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineExecutor.java b/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineExecutor.java new file mode 100644 index 000000000000..ce5c5143dce1 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineExecutor.java @@ -0,0 +1,50 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.analyzer.filter; + +/** + * Executes all linked Pipeline Tasks serially and returns + * output (if exists) from the executed logic + */ +public class FilterPipelineExecutor +{ + public static String execute(FilterPipelineTask task, String initialInput) + { + FilterPipelineTask taskPtr = task; + String result = initialInput; + + while (true) + { + FilterPipelineTask taskGeneric = taskPtr; + result = taskGeneric.process(result); + taskPtr = taskPtr.next; + + if (taskPtr == null) + { + return result; + } + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineTask.java b/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineTask.java new file mode 100644 index 000000000000..4e759547def7 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/analyzer/filter/FilterPipelineTask.java @@ -0,0 +1,60 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + + +/* + * 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.index.sai.analyzer.filter; + +/** + * A single task or set of work to process an input + * and return a single output. Maintains a link to the + * next task to be executed after itself + */ +public abstract class FilterPipelineTask +{ + private String name; + public FilterPipelineTask next; + + void setLast(String name, FilterPipelineTask last) + { + if (last == this) + throw new IllegalArgumentException("provided last task [" + last.name + "] cannot be set to itself"); + + if (this.next == null) + { + this.next = last; + this.name = name; + } + else + { + this.next.setLast(name, last); + } + } + + public abstract String process(String input); + + public String getName() + { + return name; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/BKDQueries.java b/src/java/org/apache/cassandra/index/sai/disk/BKDQueries.java new file mode 100644 index 000000000000..1a982c0ca902 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/BKDQueries.java @@ -0,0 +1,201 @@ +/* + * 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.index.sai.disk; + +import java.nio.ByteBuffer; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.disk.v1.BKDReader; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.lucene.index.PointValues.Relation; +import org.apache.lucene.util.FutureArrays; + +import static org.apache.lucene.index.PointValues.Relation.CELL_INSIDE_QUERY; + +//TODO: possible perf improvements when running on Java 9+ after replacing FutureArrays with Arrays; this needs a MR jar; +class BKDQueries +{ + private static final BKDReader.IntersectVisitor MATCH_ALL = new BKDReader.IntersectVisitor() + { + @Override + public boolean visit(byte[] packedValue) + { + return true; + } + + @Override + public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) + { + return CELL_INSIDE_QUERY; + } + }; + + static BKDReader.IntersectVisitor bkdQueryFrom(Expression expression, int numDim, int bytesPerDim) + { + if (expression.lower == null && expression.upper == null) + { + return MATCH_ALL; + } + + Bound lower = null ; + if (expression.lower != null) + { + final byte[] lowerBound = toComparableBytes(numDim, bytesPerDim, expression.lower.value.encoded, expression.validator); + lower = new Bound(lowerBound, !expression.lower.inclusive); + } + + Bound upper = null; + if (expression.upper != null) + { + final byte[] upperBound = toComparableBytes(numDim, bytesPerDim, expression.upper.value.encoded, expression.validator); + upper = new Bound(upperBound, !expression.upper.inclusive); + } + + return new RangeQueryVisitor(numDim, bytesPerDim, lower, upper); + } + + // TODO: probably move this to TypeUtil + private static byte[] toComparableBytes(int numDim, int bytesPerDim, ByteBuffer value, AbstractType type) + { + byte[] buffer = new byte[TypeUtil.fixedSizeOf(type)]; + assert buffer.length == bytesPerDim * numDim; + TypeUtil.toComparableBytes(value, type, buffer); + return buffer; + } + + private static abstract class RangeQuery implements BKDReader.IntersectVisitor + { + final int numDims; + final int bytesPerDim; + + RangeQuery(int numDims, int bytesPerDim) + { + this.numDims = numDims; + this.bytesPerDim = bytesPerDim; + } + + int compareUnsigned(byte[] packedValue, int dim, Bound bound) + { + final int offset = dim * bytesPerDim; + return FutureArrays.compareUnsigned(packedValue, offset, offset + bytesPerDim, bound.bound, offset, offset + bytesPerDim); + } + } + + private static class Bound + { + private final byte[] bound; + private final boolean exclusive; + + Bound(byte[] bound, boolean exclusive) + { + this.bound = bound; + this.exclusive = exclusive; + } + + boolean smallerThan(int cmp) + { + return cmp > 0 || (cmp == 0 && exclusive); + } + + boolean greaterThan(int cmp) + { + return cmp < 0 || (cmp == 0 && exclusive); + } + } + + private static class RangeQueryVisitor extends RangeQuery + { + private final Bound lower; + private final Bound upper; + + private RangeQueryVisitor(int numDims, int bytesPerDim, Bound lower, Bound upper) + { + super(numDims, bytesPerDim); + this.lower = lower; + this.upper = upper; + } + + @Override + public boolean visit(byte[] packedValue) + { + for (int dim = 0; dim < numDims; dim++) + { + if (lower != null) + { + int cmp = compareUnsigned(packedValue, dim, lower); + if (lower.greaterThan(cmp)) + { + // value is too low, in this dimension + return false; + } + } + + if (upper != null) + { + int cmp = compareUnsigned(packedValue, dim, upper); + if (upper.smallerThan(cmp)) + { + // value is too high, in this dimension + return false; + } + } + } + + return true; + } + + @Override + public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) + { + boolean crosses = false; + + for (int dim = 0; dim < numDims; dim++) + { + if (lower != null) + { + int maxCmp = compareUnsigned(maxPackedValue, dim, lower); + if (lower.greaterThan(maxCmp)) + return Relation.CELL_OUTSIDE_QUERY; + + int minCmp = compareUnsigned(minPackedValue, dim, lower); + crosses |= lower.greaterThan(minCmp); + } + + if (upper != null) + { + int minCmp = compareUnsigned(minPackedValue, dim, upper); + if (upper.smallerThan(minCmp)) + return Relation.CELL_OUTSIDE_QUERY; + + int maxCmp = compareUnsigned(maxPackedValue, dim, upper); + crosses |= upper.smallerThan(maxCmp); + } + } + + if (crosses) + { + return Relation.CELL_CROSSES_QUERY; + } + else + { + return Relation.CELL_INSIDE_QUERY; + } + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/ByteSliceReader.java b/src/java/org/apache/cassandra/index/sai/disk/ByteSliceReader.java new file mode 100644 index 000000000000..29ac961ab964 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/ByteSliceReader.java @@ -0,0 +1,168 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.disk; + +import java.io.IOException; + +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.util.ByteBlockPool; + +/* IndexInput that knows how to read the byte slices written + * by Posting and PostingVector. We read the bytes in + * each slice until we hit the end of that slice at which + * point we read the forwarding address of the next slice + * and then jump to it.*/ +final class ByteSliceReader extends DataInput +{ + ByteBlockPool pool; + int bufferUpto; + byte[] buffer; + public int upto; + int limit; + int level; + public int bufferOffset; + + public int endIndex; + + public void init(ByteBlockPool pool, int startIndex, int endIndex) + { + + assert endIndex - startIndex >= 0 : "startIndex=" + startIndex + " endIndex=" + endIndex; + assert startIndex >= 0; + assert endIndex >= 0; + + this.pool = pool; + this.endIndex = endIndex; + + level = 0; + bufferUpto = startIndex / ByteBlockPool.BYTE_BLOCK_SIZE; + bufferOffset = bufferUpto * ByteBlockPool.BYTE_BLOCK_SIZE; + buffer = pool.buffers[bufferUpto]; + upto = startIndex & ByteBlockPool.BYTE_BLOCK_MASK; + + final int firstSize = ByteBlockPool.LEVEL_SIZE_ARRAY[0]; + + if (startIndex + firstSize >= endIndex) + { + // There is only this one slice to read + limit = endIndex & ByteBlockPool.BYTE_BLOCK_MASK; + } + else + limit = upto + firstSize - 4; + } + + public boolean eof() + { + assert upto + bufferOffset <= endIndex; + return upto + bufferOffset == endIndex; + } + + @Override + public byte readByte() + { + assert !eof(); + assert upto <= limit; + if (upto == limit) + nextSlice(); + return buffer[upto++]; + } + + public long writeTo(DataOutput out) throws IOException + { + long size = 0; + while (true) + { + if (limit + bufferOffset == endIndex) + { + assert endIndex - bufferOffset >= upto; + out.writeBytes(buffer, upto, limit - upto); + size += limit - upto; + break; + } + else + { + out.writeBytes(buffer, upto, limit - upto); + size += limit - upto; + nextSlice(); + } + } + + return size; + } + + public void nextSlice() + { + + // Skip to our next slice + final int nextIndex = ((buffer[limit] & 0xff) << 24) + ((buffer[1 + limit] & 0xff) << 16) + ((buffer[2 + limit] & 0xff) << 8) + (buffer[3 + limit] & 0xff); + + level = ByteBlockPool.NEXT_LEVEL_ARRAY[level]; + final int newSize = ByteBlockPool.LEVEL_SIZE_ARRAY[level]; + + bufferUpto = nextIndex / ByteBlockPool.BYTE_BLOCK_SIZE; + bufferOffset = bufferUpto * ByteBlockPool.BYTE_BLOCK_SIZE; + + buffer = pool.buffers[bufferUpto]; + upto = nextIndex & ByteBlockPool.BYTE_BLOCK_MASK; + + if (nextIndex + newSize >= endIndex) + { + // We are advancing to the final slice + assert endIndex - nextIndex > 0; + limit = endIndex - bufferOffset; + } + else + { + // This is not the final slice (subtract 4 for the + // forwarding address at the end of this new slice) + limit = upto + newSize - 4; + } + } + + @Override + public void readBytes(byte[] b, int offset, int len) + { + while (len > 0) + { + final int numLeft = limit - upto; + if (numLeft < len) + { + // Read entire slice + System.arraycopy(buffer, upto, b, offset, numLeft); + offset += numLeft; + len -= numLeft; + nextSlice(); + } + else + { + // This slice is the last one + System.arraycopy(buffer, upto, b, offset, len); + upto += len; + break; + } + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/ColumnIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/ColumnIndexWriter.java new file mode 100644 index 000000000000..6c6d39f73d16 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/ColumnIndexWriter.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.index.sai.disk; + +import java.io.IOException; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.rows.Row; + +/** + * Creates an on-disk index for a given column. + */ +public interface ColumnIndexWriter +{ + /** + * Adds a row to this index. + */ + void addRow(DecoratedKey rowKey, long ssTableRowId, Row row) throws IOException; + + /** + * Builds on-disk index data structures from accumulated data, moves them all to the filesystem, and fsync created files. + */ + void flush() throws IOException; + + /** + * Aborts accumulating data. Allows to clean up resources on error. + * + * Note: Implementations should be idempotent, i.e. safe to call multiple times without producing undesirable side-effects. + */ + void abort(Throwable cause); +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/ImmutableOneDimPointValues.java b/src/java/org/apache/cassandra/index/sai/disk/ImmutableOneDimPointValues.java new file mode 100644 index 000000000000..9e8c1f16ab5a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/ImmutableOneDimPointValues.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.index.sai.disk; + +import java.io.IOException; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.lucene.codecs.MutablePointValues; +import org.apache.lucene.util.bkd.BKDWriter; + +/** + * {@link MutablePointValues} that prevents buffered points from reordering, and always skips sorting phase in Lucene + * It's the responsibility of the underlying implementation to ensure that all points are correctly sorted. + *

    + * It allows to take advantage of an optimised 1-dim writer {@link BKDWriter} + * (that is enabled only for {@link MutablePointValues}), and reduce number of times we sort point values. + */ +public class ImmutableOneDimPointValues extends MutableOneDimPointValues +{ + private final TermsIterator termEnum; + private final AbstractType termComparator; + private final byte[] scratch; + + private ImmutableOneDimPointValues(TermsIterator termEnum, AbstractType termComparator) + { + this.termEnum = termEnum; + this.termComparator = termComparator; + this.scratch = new byte[TypeUtil.fixedSizeOf(termComparator)]; + } + + public static ImmutableOneDimPointValues fromTermEnum(TermsIterator termEnum, AbstractType termComparator) + { + return new ImmutableOneDimPointValues(termEnum, termComparator); + } + + @Override + public void intersect(IntersectVisitor visitor) throws IOException + { + while (termEnum.hasNext()) + { + ByteBufferUtil.toBytes(termEnum.next().asComparableBytes(ByteComparable.Version.OSS41), scratch); + try (final PostingList postings = termEnum.postings()) + { + long segmentRowId; + while ((segmentRowId = postings.nextPosting()) != PostingList.END_OF_STREAM) + { + visitor.visit(segmentRowId, scratch); + } + } + } + } + + @Override + public int getBytesPerDimension() + { + return scratch.length; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/IndexSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/IndexSearcher.java new file mode 100644 index 000000000000..123129484c5d --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/IndexSearcher.java @@ -0,0 +1,224 @@ +/* + * 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.index.sai.disk; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.sai.SSTableContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.metrics.ColumnQueryMetrics; +import org.apache.cassandra.index.sai.metrics.QueryEventListener; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.TypeUtil; + +/** + * Abstract reader for individual segments of an on-disk index. + * + * Accepts shared resources (token/offset file readers), and uses them to perform lookups against on-disk data + * structures. + */ +public abstract class IndexSearcher implements Closeable +{ + private final LongArray.Factory rowIdToTokenFactory; + private final LongArray.Factory rowIdToOffsetFactory; + private final SSTableContext.KeyFetcher keyFetcher; + final SSTableIndex.PerIndexFiles indexFiles; + + final SegmentMetadata metadata; + + final IndexComponents indexComponents; + + IndexSearcher(Segment segment) + { + this.indexComponents = segment.indexFiles.components(); + this.rowIdToTokenFactory = segment.segmentRowIdToTokenFactory; + this.rowIdToOffsetFactory = segment.segmentRowIdToOffsetFactory; + this.keyFetcher = segment.keyFetcher; + this.indexFiles = segment.indexFiles; + this.metadata = segment.metadata; + } + + public static IndexSearcher open(boolean isString, Segment segment, ColumnQueryMetrics listener) throws IOException + { + return isString ? open(segment, (QueryEventListener.TrieIndexEventListener) listener) + : open(segment, (QueryEventListener.BKDIndexEventListener) listener); + } + + public static InvertedIndexSearcher open(Segment segment, QueryEventListener.TrieIndexEventListener listener) throws IOException + { + return new InvertedIndexSearcher(segment, listener); + } + + public static KDTreeIndexSearcher open(Segment segment, QueryEventListener.BKDIndexEventListener listener) throws IOException + { + return new KDTreeIndexSearcher(segment, listener); + } + + /** + * @return number of per-index open files attached to a sstable + */ + public static int openPerIndexFiles(AbstractType columnType) + { + return TypeUtil.isLiteral(columnType) ? InvertedIndexSearcher.openPerIndexFiles() : KDTreeIndexSearcher.openPerIndexFiles(); + } + + /** + * @return memory usage of underlying on-disk data structure + */ + public abstract long indexFileCacheSize(); + + /** + * Search on-disk index synchronously. + * + * @param expression to filter on disk index + * @param queryContext to track per sstable cache and per query metrics + * @param defer create the iterator in a deferred state + * + * @return {@link RangeIterator} that matches given expression + */ + public abstract RangeIterator search(Expression expression, SSTableQueryContext queryContext, boolean defer); + + RangeIterator toIterator(PostingList postingList, SSTableQueryContext queryContext, boolean defer) + { + if (postingList == null) + return RangeIterator.empty(); + + SearcherContext searcherContext = defer ? new DeferredSearcherContext(queryContext, postingList.peekable()) + : new DirectSearcherContext(queryContext, postingList.peekable()); + + if (searcherContext.noOverlap) + return RangeIterator.empty(); + + RangeIterator iterator = new PostingListRangeIterator(searcherContext, keyFetcher, indexComponents); + + return iterator; + } + + public abstract class SearcherContext + { + long minToken; + long maxToken; + long maxPartitionOffset; + boolean noOverlap; + final LongArray segmentRowIdToToken; + final LongArray segmentRowIdToOffset; + final SSTableQueryContext context; + final PostingList.PeekablePostingList postingList; + + SearcherContext(SSTableQueryContext context, PostingList.PeekablePostingList postingList) + { + this.context = context; + this.postingList = postingList; + + // startingIndex of 0 means `findTokenRowId` should search all tokens in the segment. + this.segmentRowIdToToken = new LongArray.DeferredLongArray(() -> rowIdToTokenFactory.openTokenReader(0, context)); + this.segmentRowIdToOffset = new LongArray.DeferredLongArray(() -> rowIdToOffsetFactory.open()); + + minToken = calculateMinimumToken(); + + // use segment's metadata for the range iterator, may not be accurate, but should not matter to performance. + maxToken = metadata.maxKey.isMinimum() + ? toLongToken(DatabaseDescriptor.getPartitioner().getMaximumToken()) + : toLongToken(metadata.maxKey); + + maxPartitionOffset = Long.MAX_VALUE; + } + + long minToken() + { + return minToken; + } + + long maxToken() + { + return maxToken; + } + + abstract long calculateMinimumToken(); + + abstract long count(); + + } + + public class DirectSearcherContext extends SearcherContext + { + DirectSearcherContext(SSTableQueryContext context, PostingList.PeekablePostingList postingList) + { + super(context, postingList); + } + + @Override + long calculateMinimumToken() + { + // Use the first row id's token as min + return this.segmentRowIdToToken.get(postingList.peek()); + } + + @Override + long count() + { + return postingList.size(); + } + } + + public class DeferredSearcherContext extends SearcherContext + { + DeferredSearcherContext(SSTableQueryContext context, PostingList.PeekablePostingList postingList) + { + super(context, postingList); + } + + @Override + long calculateMinimumToken() + { + // Use the segments min key min + return toLongToken(metadata.minKey); + } + + @Override + long count() + { + return metadata.numRows; + } + } + + private static long toLongToken(DecoratedKey key) + { + return toLongToken(key.getToken()); + } + + private static long toLongToken(ByteBuffer key) + { + return toLongToken(DatabaseDescriptor.getPartitioner().getToken(key)); + } + + private static long toLongToken(Token token) + { + return (long) token.getTokenValue(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/IndexWriterConfig.java b/src/java/org/apache/cassandra/index/sai/disk/IndexWriterConfig.java new file mode 100644 index 000000000000..4a1dbec3d256 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/IndexWriterConfig.java @@ -0,0 +1,141 @@ +/* + * 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.index.sai.disk; + +import java.util.Map; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.sai.utils.TypeUtil; + +/** + * Per-index config for storage-attached index writers. + */ +public class IndexWriterConfig +{ + public static final String POSTING_LIST_LVL_MIN_LEAVES = "bkd_postings_min_leaves"; + public static final String POSTING_LIST_LVL_SKIP_OPTION = "bkd_postings_skip"; + + private static final int DEFAULT_POSTING_LIST_MIN_LEAVES = 64; + private static final int DEFAULT_POSTING_LIST_LVL_SKIP = 3; + + private static final IndexWriterConfig EMPTY_CONFIG = new IndexWriterConfig(null, -1, -1); + + /** + * Fully qualified index name, in the format ".

    .". + */ + private final String indexName; + + /** + * Skip, or the sampling interval, for selecting a bkd tree level that is eligible for an auxiliary posting list. + * Sampling starts from 0, but bkd tree root node is at level 1. For skip = 4, eligible levels are 4, 8, 12, etc (no + * level 0, because there is no node at level 0). + */ + private final int bkdPostingsSkip; + + /** + * Min. number of reachable leaves for a given node to be eligible for an auxiliary posting list. + */ + private final int bkdPostingsMinLeaves; + + public IndexWriterConfig(String indexName, int bkdPostingsSkip, int bkdPostingsMinLeaves) + { + this.indexName = indexName; + this.bkdPostingsSkip = bkdPostingsSkip; + this.bkdPostingsMinLeaves = bkdPostingsMinLeaves; + } + + public String getIndexName() + { + return indexName; + } + + public int getBkdPostingsMinLeaves() + { + return bkdPostingsMinLeaves; + } + + public int getBkdPostingsSkip() + { + return bkdPostingsSkip; + } + + public static IndexWriterConfig fromOptions(String indexName, AbstractType type, Map options) + { + int minLeaves = DEFAULT_POSTING_LIST_MIN_LEAVES; + int skip = DEFAULT_POSTING_LIST_LVL_SKIP; + + if (options.get(POSTING_LIST_LVL_MIN_LEAVES) != null || options.get(POSTING_LIST_LVL_SKIP_OPTION) != null) + { + if (TypeUtil.isLiteral(type)) + { + throw new InvalidRequestException(String.format("CQL type %s cannot have auxiliary posting lists on index %s.", type.asCQL3Type(), indexName)); + } + + for (Map.Entry entry : options.entrySet()) + { + switch (entry.getKey()) + { + case POSTING_LIST_LVL_MIN_LEAVES: + { + minLeaves = Integer.parseInt(entry.getValue()); + + if (minLeaves < 1) + { + throw new InvalidRequestException(String.format("Posting list min. leaves count can't be less than 1 on index %s.", indexName)); + } + + break; + } + + case POSTING_LIST_LVL_SKIP_OPTION: + { + skip = Integer.parseInt(entry.getValue()); + + if (skip < 1) + { + throw new InvalidRequestException(String.format("Posting list skip can't be less than 1 on index %s.", indexName)); + } + + break; + } + } + } + } + + return new IndexWriterConfig(indexName, skip, minLeaves); + } + + public static IndexWriterConfig defaultConfig(String indexName) + { + return new IndexWriterConfig(indexName, DEFAULT_POSTING_LIST_LVL_SKIP, DEFAULT_POSTING_LIST_MIN_LEAVES); + } + + public static IndexWriterConfig emptyConfig() + { + return EMPTY_CONFIG; + } + + @Override + public String toString() + { + return String.format("IndexWriterConfig{%s=%d, %s=%d}", + POSTING_LIST_LVL_SKIP_OPTION, bkdPostingsSkip, + POSTING_LIST_LVL_MIN_LEAVES, bkdPostingsMinLeaves); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/InvertedIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/InvertedIndexSearcher.java new file mode 100644 index 000000000000..96763f86ec37 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/InvertedIndexSearcher.java @@ -0,0 +1,120 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.Map; + +import com.google.common.base.MoreObjects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.TermsReader; +import org.apache.cassandra.index.sai.metrics.MulticastQueryEventListeners; +import org.apache.cassandra.index.sai.metrics.QueryEventListener; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.lucene.util.RamUsageEstimator; + +/** + * Executes {@link Expression}s against the trie-based terms dictionary for an individual index segment. + */ +public class InvertedIndexSearcher extends IndexSearcher +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final TermsReader reader; + private final QueryEventListener.TrieIndexEventListener perColumnEventListener; + + InvertedIndexSearcher(Segment segment, QueryEventListener.TrieIndexEventListener listener) throws IOException + { + super(segment); + + long root = metadata.getIndexRoot(indexComponents.termsData); + assert root >= 0; + + perColumnEventListener = listener; + + Map map = metadata.componentMetadatas.get(IndexComponents.NDIType.TERMS_DATA).attributes; + String footerPointerString = map.get(SAICodecUtils.FOOTER_POINTER); + long footerPointer = footerPointerString == null ? -1 : Long.parseLong(footerPointerString); + + reader = new TermsReader(indexComponents, + indexFiles.termsData().sharedCopy(), + indexFiles.postingLists().sharedCopy(), + root, footerPointer); + } + + @Override + public long indexFileCacheSize() + { + // trie has no pre-allocated memory. + // TODO: Is this still the case now the trie isn't using the chunk cache? + return 0; + } + + @Override + @SuppressWarnings("resource") + public RangeIterator search(Expression exp, SSTableQueryContext context, boolean defer) + { + if (logger.isTraceEnabled()) + logger.trace(indexComponents.logMessage("Searching on expression '{}'..."), exp); + + if (!exp.getOp().isEquality()) + throw new IllegalArgumentException(indexComponents.logMessage("Unsupported expression: " + exp)); + + final ByteComparable term = ByteComparable.fixedLength(exp.lower.value.encoded); + QueryEventListener.TrieIndexEventListener listener = MulticastQueryEventListeners.of(context.queryContext, perColumnEventListener); + + PostingList postingList = defer ? new PostingList.DeferredPostingList(() -> reader.exactMatch(term, listener, context.queryContext)) + : reader.exactMatch(term, listener, context.queryContext); + return toIterator(postingList, context, defer); + } + + public static int openPerIndexFiles() + { + return TermsReader.openPerIndexFiles(); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("indexComponents", indexComponents) + .add("diskSize", RamUsageEstimator.humanReadableUnits(indexComponents.sizeOfPerColumnComponents())) + .toString(); + } + + @Override + public void close() + { + reader.close(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/KDTreeIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/KDTreeIndexSearcher.java new file mode 100644 index 000000000000..ef7e1e141d6a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/KDTreeIndexSearcher.java @@ -0,0 +1,115 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; + +import com.google.common.base.MoreObjects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.disk.v1.BKDReader; +import org.apache.cassandra.index.sai.metrics.MulticastQueryEventListeners; +import org.apache.cassandra.index.sai.metrics.QueryEventListener; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.index.sai.disk.BKDQueries.bkdQueryFrom; + +/** + * Executes {@link Expression}s against the kd-tree for an individual index segment. + */ +public class KDTreeIndexSearcher extends IndexSearcher +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final BKDReader bkdReader; + private final QueryEventListener.BKDIndexEventListener perColumnEventListener; + + KDTreeIndexSearcher(Segment segment, QueryEventListener.BKDIndexEventListener listener) throws IOException + { + super(segment); + + final long bkdPosition = metadata.getIndexRoot(indexComponents.kdTree); + assert bkdPosition >= 0; + final long postingsPosition = metadata.getIndexRoot(indexComponents.kdTreePostingLists); + assert postingsPosition >= 0; + + bkdReader = new BKDReader(indexFiles.components(), + indexFiles.kdtree().sharedCopy(), + bkdPosition, + indexFiles.kdtreePostingLists().sharedCopy(), + postingsPosition); + perColumnEventListener = listener; + + } + + @Override + public long indexFileCacheSize() + { + return bkdReader.memoryUsage(); + } + + @Override + @SuppressWarnings("resource") + public RangeIterator search(Expression exp, SSTableQueryContext context, boolean defer) + { + if (logger.isTraceEnabled()) + logger.trace(indexComponents.logMessage("Searching on expression '{}'..."), exp); + + if (exp.getOp().isEqualityOrRange()) + { + final BKDReader.IntersectVisitor query = bkdQueryFrom(exp, bkdReader.getNumDimensions(), bkdReader.getBytesPerDimension()); + QueryEventListener.BKDIndexEventListener listener = MulticastQueryEventListeners.of(context.queryContext, perColumnEventListener); + + PostingList postingList = defer ? new PostingList.DeferredPostingList(() -> bkdReader.intersect(query, listener, context.queryContext)) + : bkdReader.intersect(query, listener, context.queryContext); + return toIterator(postingList, context, defer); + } + else + { + throw new IllegalArgumentException(indexComponents.logMessage(indexComponents.logMessage("Unsupported expression during index query: " + exp))); + } + } + + public static int openPerIndexFiles() + { + return BKDReader.openPerIndexFiles(); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("indexComponents", indexComponents) + .add("diskSize", FBUtilities.prettyPrintMemory(indexComponents.sizeOfPerColumnComponents())) + .add("count", bkdReader.getPointCount()) + .add("numDimensions", bkdReader.getNumDimensions()) + .add("bytesPerDimension", bkdReader.getBytesPerDimension()) + .toString(); + } + + @Override + public void close() + { + bkdReader.close(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/MemtableIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/MemtableIndexWriter.java new file mode 100644 index 000000000000..254a5cd89251 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/MemtableIndexWriter.java @@ -0,0 +1,186 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.Collections; +import java.util.Iterator; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.carrotsearch.hppc.IntArrayList; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.InvertedIndexWriter; +import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; +import org.apache.cassandra.index.sai.disk.v1.NumericIndexWriter; +import org.apache.cassandra.index.sai.memory.MemtableIndex; +import org.apache.cassandra.index.sai.memory.RowMapping; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * Column index writer that flushes indexed data directly from the corresponding Memtable index, without buffering index + * data in memory. + */ +public class MemtableIndexWriter implements ColumnIndexWriter +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final MemtableIndex memtable; + private final RowMapping rowMapping; + private final ColumnContext context; + private final Descriptor descriptor; + private final IndexComponents indexComponents; + + public MemtableIndexWriter(MemtableIndex memtable, + Descriptor descriptor, + ColumnContext context, + RowMapping rowMapping, + CompressionParams compressionParams) + { + assert rowMapping != null && rowMapping != RowMapping.DUMMY : "Row mapping must exist during FLUSH."; + + this.memtable = memtable; + this.rowMapping = rowMapping; + this.context = context; + this.descriptor = descriptor; + + this.indexComponents = IndexComponents.create(context.getIndexName(), descriptor, compressionParams); + } + + @Override + public void addRow(DecoratedKey rowKey, long ssTableRowId, Row row) + { + // Memtable indexes are flushed directly to disk with the aid of a mapping between primary + // keys and row IDs in the flushing SSTable. This writer, therefore, does nothing in + // response to the flushing of individual rows. + } + + @Override + public void abort(Throwable cause) + { + logger.warn(context.logMessage("Aborting index memtable flush for {}..."), descriptor, cause); + indexComponents.deleteColumnIndex(); + } + + @Override + public void flush() throws IOException + { + long start = System.nanoTime(); + + try + { + if (!rowMapping.hasRows() || (memtable == null) || memtable.isEmpty()) + { + logger.debug(context.logMessage("No indexed rows to flush from SSTable {}."), descriptor); + // Write a completion marker even though we haven't written anything to the index + // so we won't try to build the index again for the SSTable + indexComponents.createColumnCompletionMarker(); + return; + } + + final DecoratedKey minKey = rowMapping.minKey; + final DecoratedKey maxKey = rowMapping.maxKey; + + final Iterator> iterator = rowMapping.merge(memtable); + + try (MemtableTermsIterator terms = new MemtableTermsIterator(memtable.getMinTerm(), memtable.getMaxTerm(), iterator)) + { + long cellCount = flush(minKey, maxKey, context.getValidator(), terms, rowMapping.maxSegmentRowId); + + indexComponents.createColumnCompletionMarker(); + + context.getIndexMetrics().memtableIndexFlushCount.inc(); + + long durationMillis = Math.max(1, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); + + if (logger.isTraceEnabled()) + { + logger.trace(context.logMessage("Flushed {} Memtable index cells for {} in {} ms."), cellCount, descriptor, durationMillis); + } + + context.getIndexMetrics().memtableFlushCellsPerSecond.update((long) (cellCount * 1000.0 / durationMillis)); + } + } + catch (Throwable t) + { + logger.error(context.logMessage("Error while flushing index {}"), t.getMessage(), t); + context.getIndexMetrics().memtableIndexFlushErrors.inc(); + + throw t; + } + } + + private long flush(DecoratedKey minKey, DecoratedKey maxKey, AbstractType termComparator, MemtableTermsIterator terms, int maxSegmentRowId) throws IOException + { + long numRows; + SegmentMetadata.ComponentMetadataMap indexMetas; + + if (TypeUtil.isLiteral(termComparator)) + { + try (InvertedIndexWriter writer = new InvertedIndexWriter(indexComponents, false)) + { + indexMetas = writer.writeAll(terms); + numRows = writer.getPostingsCount(); + } + } + else + { + try (NumericIndexWriter writer = new NumericIndexWriter(indexComponents, + TypeUtil.fixedSizeOf(termComparator), + maxSegmentRowId, + // Due to stale entries in IndexMemtable, we may have more indexed rows than num of rowIds. + Integer.MAX_VALUE, + context.getIndexWriterConfig(), + false)) + { + indexMetas = writer.writeAll(ImmutableOneDimPointValues.fromTermEnum(terms, termComparator)); + numRows = writer.getPointCount(); + } + } + + // If no rows were written we need to delete any created column index components + // so that the index is correctly identified as being empty (only having a completion marker) + if (numRows == 0) + { + indexComponents.deleteColumnIndex(); + return 0; + } + + // During index memtable flush, the data is sorted based on terms. + SegmentMetadata metadata = new SegmentMetadata(0, numRows, terms.getMinSSTableRowId(), terms.getMaxSSTableRowId(), + minKey, maxKey, terms.getMinTerm(), terms.getMaxTerm(), indexMetas); + + try (MetadataWriter writer = new MetadataWriter(indexComponents.createOutput(indexComponents.meta))) + { + SegmentMetadata.write(writer, Collections.singletonList(metadata), null); + } + + return numRows; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/MemtableTermsIterator.java b/src/java/org/apache/cassandra/index/sai/disk/MemtableTermsIterator.java new file mode 100644 index 000000000000..3a99639064c3 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/MemtableTermsIterator.java @@ -0,0 +1,134 @@ +/* + * 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.index.sai.disk; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +import com.google.common.base.Preconditions; + +import com.carrotsearch.hppc.IntArrayList; +import com.carrotsearch.hppc.cursors.IntCursor; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * Iterator over a token range bounded segment of a Memtable index. Used to flush Memtable index segments to disk. + */ +public class MemtableTermsIterator implements TermsIterator +{ + private final ByteBuffer minTerm; + private final ByteBuffer maxTerm; + private final Iterator> iterator; + + private Pair current; + + private long maxSSTableRowId = -1; + private long minSSTableRowId = Long.MAX_VALUE; + + public MemtableTermsIterator(ByteBuffer minTerm, + ByteBuffer maxTerm, + Iterator> iterator) + { + Preconditions.checkArgument(iterator != null); + this.minTerm = minTerm; + this.maxTerm = maxTerm; + this.iterator = iterator; + } + + @Override + public ByteBuffer getMinTerm() + { + return minTerm; + } + + @Override + public ByteBuffer getMaxTerm() + { + return maxTerm; + } + + @Override + public void close() {} + + @Override + public PostingList postings() + { + //TODO Confirm that this can stay an IntArray post DSP-19608 + final IntArrayList list = current.right; + + assert list.size() > 0; + + final int minSegmentRowID = list.get(0); + final int maxSegmentRowID = list.get(list.size() - 1); + + minSSTableRowId = Math.min(minSSTableRowId, minSegmentRowID); + maxSSTableRowId = Math.max(maxSSTableRowId, maxSegmentRowID); + + final Iterator it = list.iterator(); + + return new PostingList() + { + @Override + public long nextPosting() + { + if (!it.hasNext()) + { + return END_OF_STREAM; + } + + return it.next().value; + } + + @Override + public long size() + { + return list.size(); + } + + @Override + public long advance(long targetRowID) + { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + + @Override + public ByteComparable next() + { + current = iterator.next(); + return current.left; + } + + long getMaxSSTableRowId() + { + return maxSSTableRowId; + } + + long getMinSSTableRowId() + { + return minSSTableRowId; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/MergeOneDimPointValues.java b/src/java/org/apache/cassandra/index/sai/disk/MergeOneDimPointValues.java new file mode 100644 index 000000000000..f7bbc8a75cbf --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/MergeOneDimPointValues.java @@ -0,0 +1,152 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.disk.v1.BKDReader; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.lucene.util.PriorityQueue; +import org.apache.lucene.util.bkd.BKDWriter; + +/** + * {@link MutableOneDimPointValues} that prevents buffered points from reordering, and always skips sorting phase in Lucene + * It's the responsibility of the underlying implementation to ensure that all points are correctly sorted. + *

    + * It allows to take advantage of an optimised 1-dim writer {@link BKDWriter} + * (that is enabled only for {@link MutableOneDimPointValues}), and reduce number of times we sort point values. + */ +public class MergeOneDimPointValues extends MutableOneDimPointValues +{ + private static final byte[] EMPTY = new byte[0]; + + private final byte[] scratch; + private final MergeQueue queue; + + public long minRowID = Long.MAX_VALUE; + public long maxRowID = Long.MIN_VALUE; + public long numRows = 0; + + public MergeOneDimPointValues(List iterators, AbstractType termComparator) throws IOException + { + queue = new MergeQueue(iterators.size()); + this.scratch = new byte[TypeUtil.fixedSizeOf(termComparator)]; + for (BKDReader.IteratorState iterator : iterators) + { + if (iterator.hasNext()) + { + queue.add(iterator); + } + } + } + + @VisibleForTesting + public MergeOneDimPointValues(List iterators, int bytesPerDim) throws IOException + { + queue = new MergeQueue(iterators.size()); + this.scratch = new byte[bytesPerDim]; + for (BKDReader.IteratorState iterator : iterators) + { + if (iterator.hasNext()) + { + queue.add(iterator); + } + } + } + + public long getMinRowID() + { + return minRowID; + } + + public long getMaxRowID() + { + return maxRowID; + } + + public long getNumRows() + { + return numRows; + } + + @Override + @SuppressWarnings("resource") + public void intersect(IntersectVisitor visitor) throws IOException + { + while (queue.size() != 0) + { + final BKDReader.IteratorState reader = queue.top(); + if (reader.hasNext()) + { + final long rowID = reader.next(); + + minRowID = Math.min(minRowID, rowID); + maxRowID = Math.max(maxRowID, rowID); + numRows++; + + visitor.visit(rowID, reader.scratch); + + if (reader.hasNext()) + { + queue.updateTop(); + } + else + { + queue.pop(); + } + } + else + { + // iterator is exhausted + queue.pop(); + } + } + } + + @Override + public int getBytesPerDimension() + { + return scratch.length; + } + + private static class MergeQueue extends PriorityQueue + { + public MergeQueue(int maxSize) + { + super(maxSize); + } + + @Override + public boolean lessThan(BKDReader.IteratorState a, BKDReader.IteratorState b) + { + assert a != b; + + int cmp = a.compareTo(b); + + if (cmp < 0) + { + return true; + } + else return false; + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/MergingIterator.java b/src/java/org/apache/cassandra/index/sai/disk/MergingIterator.java new file mode 100644 index 000000000000..7a82014bb76a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/MergingIterator.java @@ -0,0 +1,172 @@ +/* + * 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.index.sai.disk; + + +import java.util.Iterator; +import java.util.NoSuchElementException; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.lucene.util.PriorityQueue; + +public final class MergingIterator implements Iterator +{ + private ByteComparable current; + private final TermMergeQueue queue; + final SubIterator[] top; + private final boolean removeDuplicates = true; + private int numTop; + + @SuppressWarnings({ "unchecked", "rawtypes" }) + public MergingIterator(AbstractType type, Iterator... iterators) + { + queue = new TermMergeQueue(iterators.length, type); + top = new SubIterator[iterators.length]; + int index = 0; + for (Iterator iterator : iterators) + { + if (iterator.hasNext()) + { + SubIterator sub = new SubIterator(); + sub.current = iterator.next(); + sub.iterator = iterator; + sub.index = index++; + queue.add(sub); + } + } + } + + @Override + public boolean hasNext() + { + if (queue.size() > 0) + { + return true; + } + + for (int i = 0; i < numTop; i++) + { + if (top[i].iterator.hasNext()) + { + return true; + } + } + return false; + } + + public int getNumTop() + { + return numTop; + } + + @Override + public ByteComparable next() + { + // restore queue + pushTop(); + + // gather equal top elements + if (queue.size() > 0) + { + pullTop(); + } + else + { + current = null; + } + if (current == null) + { + throw new NoSuchElementException(); + } + return current; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + private void pullTop() + { + assert numTop == 0; + top[numTop++] = queue.pop(); + if (removeDuplicates) + { + // extract all subs from the queue that have the same top element + while (queue.size() != 0 + && ByteComparable.compare(queue.top().current, top[0].current, ByteComparable.Version.OSS41) == 0) + { + top[numTop++] = queue.pop(); + } + } + current = top[0].current; + } + + private void pushTop() + { + // call next() on each top, and put back into queue + for (int i = 0; i < numTop; i++) + { + if (top[i].iterator.hasNext()) + { + top[i].current = top[i].iterator.next(); + queue.add(top[i]); + } + else + { + // no more elements + top[i].current = null; + } + } + numTop = 0; + } + + public static class SubIterator + { + Iterator iterator; + ByteComparable current; + int index; + } + + private static class TermMergeQueue extends PriorityQueue + { + final AbstractType type; + + TermMergeQueue(int size, AbstractType type) + { + super(size); + this.type = type; + } + + @Override + protected boolean lessThan(SubIterator a, SubIterator b) + { + final int cmp = ByteComparable.compare(a.current, b.current, ByteComparable.Version.OSS41); + + if (cmp != 0) + { + return cmp < 0; + } + else + { + return a.index < b.index; + } + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/MutableOneDimPointValues.java b/src/java/org/apache/cassandra/index/sai/disk/MutableOneDimPointValues.java new file mode 100644 index 000000000000..3c42b2b74fd7 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/MutableOneDimPointValues.java @@ -0,0 +1,112 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; + +import org.apache.lucene.codecs.MutablePointValues; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.util.BytesRef; + +public abstract class MutableOneDimPointValues extends MutablePointValues +{ + private static final byte[] EMPTY = new byte[0]; + + abstract public void intersect(IntersectVisitor visitor) throws IOException; + + @Override + public int getDocCount() + { + throw new UnsupportedOperationException(); + } + + @Override + public long size() + { + // hack to skip sorting in Lucene + return 1; + } + + @Override + public void getValue(int i, BytesRef packedValue) + { + // no-op + } + + @Override + public byte getByteAt(int i, int k) + { + return 0; + } + + @Override + public int getDocID(int i) + { + return 0; + } + + @Override + public void swap(int i, int j) + { + throw new IllegalStateException("unexpected sorting"); + } + + @Override + public void intersect(PointValues.IntersectVisitor visitor) throws IOException + { + throw new UnsupportedOperationException(); + } + + @Override + public long estimatePointCount(PointValues.IntersectVisitor visitor) + { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getMinPackedValue() + { + return EMPTY; + } + + @Override + public byte[] getMaxPackedValue() + { + return EMPTY; + } + + @Override + public int getNumDimensions() + { + return 1; + } + + @Override + public int getBytesPerDimension() + { + return 0; + } + + public interface IntersectVisitor + { + /** Called for all documents in a leaf cell that crosses the query. The consumer + * should scrutinize the packedValue to decide whether to accept it. In the 1D case, + * values are visited in increasing order, and in the case of ties, in increasing + * docID order. */ + void visit(long docID, byte[] packedValue) throws IOException; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/OnDiskKeyProducer.java b/src/java/org/apache/cassandra/index/sai/disk/OnDiskKeyProducer.java new file mode 100644 index 000000000000..6361c6bfac6f --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/OnDiskKeyProducer.java @@ -0,0 +1,110 @@ +/* + * 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.index.sai.disk; + +import java.util.Collections; +import java.util.Iterator; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.Iterators; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.SSTableContext; +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.io.util.RandomAccessReader; + +/** + * A reusable {@link Token} that lazily materializes an iterator of {@link DecoratedKey} from disk. + */ +@NotThreadSafe +public class OnDiskKeyProducer +{ + public static final long NO_OFFSET = -1; + + private final SSTableContext.KeyFetcher keyFetcher; + private final RandomAccessReader reader; + private final LongArray segmentRowIdToOffset; + + private final long maxPartitionOffset; + + private long lastOffset = NO_OFFSET; + + public OnDiskKeyProducer(SSTableContext.KeyFetcher keyFetcher, RandomAccessReader reader, LongArray segmentRowIdToOffset, long maxPartitionOffset) + { + this.keyFetcher = keyFetcher; + this.reader = reader; + this.segmentRowIdToOffset = segmentRowIdToOffset; + this.maxPartitionOffset = maxPartitionOffset; + } + + public Token produceToken(long token, long segmentRowId) + { + return new OnDiskToken(token, segmentRowId); + } + + /** + * Used to remove duplicated key offset due rows sharing the same key offset in wide partition schema. + */ + private long getKeyOffset(long segmentRowId) + { + long offset = segmentRowIdToOffset.get(segmentRowId); + + if (offset == lastOffset) + { + return NO_OFFSET; + } + + // Due to ZCS, index files may still contain partition offsets that are not part of partial SSTable. + if (offset > maxPartitionOffset) + { + return NO_OFFSET; + } + + // Catalog the last offset if it's valid: + lastOffset = offset; + + return offset; + } + + public class OnDiskToken extends Token + { + private final long segmentRowId; + + public OnDiskToken(long token, long segmentRowId) + { + super(token); + this.segmentRowId = segmentRowId; + } + + @Override + public Iterator keys() + { + long keyOffset = getKeyOffset(segmentRowId); + DecoratedKey key = keyFetcher.apply(reader, keyOffset); + return key == null ? Collections.emptyIterator() : Iterators.singletonIterator(key); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this).add("token", token).add("lastOffset", lastOffset).toString(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/PostingList.java b/src/java/org/apache/cassandra/index/sai/disk/PostingList.java new file mode 100644 index 000000000000..5dfefaf6b933 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/PostingList.java @@ -0,0 +1,197 @@ +/* + * 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.index.sai.disk; + +import java.io.Closeable; +import java.io.IOException; +import java.util.function.Supplier; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.cassandra.utils.Throwables; + +/** + * Interface for advancing on and consuming a posting list. + */ +//TODO Need to check int and long usage throughout this post DSP-19608 +@NotThreadSafe +public interface PostingList extends Closeable +{ + long OFFSET_NOT_FOUND = -1; + long END_OF_STREAM = Long.MAX_VALUE; + + @Override + default void close() throws IOException {} + + /** + * Retrieves the next segment row ID, not including row IDs that have been returned by {@link #advance(long)}. + * + * @return next segment row ID + */ + long nextPosting() throws IOException; + + long size(); + + /** + * Advances to the first row ID beyond the current that is greater than or equal to the + * target, and returns that row ID. Exhausts the iterator and returns {@link #END_OF_STREAM} if + * the target is greater than the highest row ID. + * + * Note: Callers must use the return value of this method before calling {@link #nextPosting()}, as calling + * that method will return the next posting, not the one to which we have just advanced. + * + * @param targetRowID target row ID to advance to + * + * @return first segment row ID which is >= the target row ID or {@link PostingList#END_OF_STREAM} if one does not exist + */ + long advance(long targetRowID) throws IOException; + + /** + * @return peekable wrapper of current posting list + */ + default PeekablePostingList peekable() + { + return new PeekablePostingList(this); + } + + class DeferredPostingList implements PostingList + { + private Supplier supplier; + private PostingList postingList; + private boolean opened = false; + + public DeferredPostingList(Supplier supplier) + { + this.supplier = supplier; + } + + @Override + public long nextPosting() throws IOException + { + open(); + return postingList == null ? END_OF_STREAM : postingList.nextPosting(); + } + + @Override + public long size() + { + open(); + return postingList == null ? 0 : postingList.size(); + } + + @Override + public long advance(long targetRowID) throws IOException + { + open(); + return postingList == null ? END_OF_STREAM : postingList.advance(targetRowID); + } + + @Override + public void close() throws IOException + { + if (opened && (postingList != null)) + postingList.close(); + } + + private void open() + { + if (!opened) + { + postingList = supplier.get(); + opened = true; + } + } + } + + public static class PeekablePostingList implements PostingList + { + private final PostingList wrapped; + + private boolean peeked = false; + private long next; + + public PeekablePostingList(PostingList wrapped) + { + this.wrapped = wrapped; + } + + public long peek() + { + if (peeked) + return next; + + try + { + peeked = true; + return next = wrapped.nextPosting(); + } + catch (IOException e) + { + throw Throwables.cleaned(e); + } + } + + public long advanceWithoutConsuming(long targetRowID) throws IOException + { + if (peek() == END_OF_STREAM) + return END_OF_STREAM; + + if (peek() >= targetRowID) + return peek(); + + peeked = true; + next = wrapped.advance(targetRowID); + return next; + } + + @Override + public long nextPosting() throws IOException + { + if (peeked) + { + peeked = false; + return next; + } + return wrapped.nextPosting(); + } + + @Override + public long size() + { + return wrapped.size(); + } + + @Override + public long advance(long targetRowID) throws IOException + { + if (peeked && next >= targetRowID) + { + peeked = false; + return next; + } + + peeked = false; + return wrapped.advance(targetRowID); + } + + @Override + public void close() throws IOException + { + wrapped.close(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/PostingListRangeIterator.java b/src/java/org/apache/cassandra/index/sai/disk/PostingListRangeIterator.java new file mode 100644 index 000000000000..0077dacad4ed --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/PostingListRangeIterator.java @@ -0,0 +1,199 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.base.Stopwatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.index.sai.SSTableContext; +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.utils.AbortedOperationException; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.utils.Throwables; + +/** + * A range iterator based on {@link PostingList}. + * + *

      + *
    1. fetch next segment row id from posting list or skip to specific segment row id if {@link #skipTo(Long)} is called
    2. + *
    3. produce a {@link OnDiskKeyProducer.OnDiskToken} from {@link OnDiskKeyProducer#produceToken(long, int)} which is used + * to avoid fetching duplicated keys due to partition-level indexing on wide partition schema. + *
      + * Note: in order to reduce disk access in multi-index query, partition keys will only be fetched for intersected tokens + * in {@link org.apache.cassandra.index.sai.plan.StorageAttachedIndexSearcher}. + *
    4. + *
    + * + */ + +@NotThreadSafe +public class PostingListRangeIterator extends RangeIterator +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final Stopwatch timeToExhaust = Stopwatch.createStarted(); + private final SSTableQueryContext queryContext; + private final IndexComponents components; + + private final PostingList postingList; + private final SSTableContext.KeyFetcher keyFetcher; + private final IndexSearcher.SearcherContext context; + private final LongArray segmentRowIdToToken; + private final LongArray segmentRowIdToOffset; + + private RandomAccessReader keyReader = null; + private OnDiskKeyProducer producer = null; + + private boolean opened = false; + private boolean needsSkipping = false; + private long skipToToken = Long.MIN_VALUE; + + + /** + * Create a direct PostingListRangeIterator where the underlying PostingList is materialised + * immediately so the posting list size can be used. + */ + public PostingListRangeIterator(IndexSearcher.SearcherContext context, + SSTableContext.KeyFetcher keyFetcher, + IndexComponents components) + { + super(context.minToken(), context.maxToken(), context.count()); + + this.keyFetcher = keyFetcher; + this.segmentRowIdToToken = context.segmentRowIdToToken; + this.segmentRowIdToOffset = context.segmentRowIdToOffset; + this.postingList = context.postingList; + this.context = context; + this.queryContext = context.context; + this.components = components; + } + + @Override + protected void performSkipTo(Long nextToken) + { + if (skipToToken >= nextToken) + return; + + skipToToken = nextToken; + needsSkipping = true; + } + + @Override + protected Token computeNext() + { + try + { + queryContext.queryContext.checkpoint(); + + if (!opened) + open(); + + // just end the iterator if we don't have a postingList or current segment is skipped + if (exhausted()) + return endOfData(); + + long segmentRowId = getNextSegmentRowId(); + if (segmentRowId == PostingList.END_OF_STREAM) + return endOfData(); + + return getNextToken(segmentRowId); + } + catch (Throwable t) + { + //TODO We aren't tidying up resources here + if (!(t instanceof AbortedOperationException)) + logger.error(components.logMessage("Unable to provide next token!"), t); + + throw Throwables.cleaned(t); + } + } + + @Override + public void close() throws IOException + { + if (logger.isTraceEnabled()) + { + final long exhaustedInMills = timeToExhaust.stop().elapsed(TimeUnit.MILLISECONDS); + logger.trace(components.logMessage("PostinListRangeIterator exhausted after {} ms"), exhaustedInMills); + } + + postingList.close(); + FileUtils.closeQuietly(segmentRowIdToToken, segmentRowIdToOffset, keyReader); + } + + private void open() + { + this.keyReader = keyFetcher.createReader(); + this.producer = new OnDiskKeyProducer(keyFetcher, keyReader, segmentRowIdToOffset, context.maxPartitionOffset); + opened = true; + } + + private boolean exhausted() + { + return needsSkipping && skipToToken > getMaximum(); + } + + /** + * reads the next row ID from the underlying posting list, potentially skipping to get there. + */ + private long getNextSegmentRowId() throws IOException + { + if (needsSkipping) + { + int targetRowID = Math.toIntExact(segmentRowIdToToken.findTokenRowID(skipToToken)); + // skipToToken is larger than max token in token file + if (targetRowID < 0) + { + return PostingList.END_OF_STREAM; + } + + long segmentRowId = postingList.advance(targetRowID); + + needsSkipping = false; + return segmentRowId; + } + else + { + return postingList.nextPosting(); + } + } + + /** + * takes a segment row ID and produces a {@link Token} for its partition key. + */ + private Token getNextToken(long segmentRowId) + { + assert segmentRowId != PostingList.END_OF_STREAM; + + long tokenValue = segmentRowIdToToken.get(segmentRowId); + + // Used to remove duplicated key offset + return producer.produceToken(tokenValue, segmentRowId); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/QueryEventListeners.java b/src/java/org/apache/cassandra/index/sai/disk/QueryEventListeners.java new file mode 100644 index 000000000000..ba6a80265c66 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/QueryEventListeners.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.index.sai.disk; + +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.index.sai.metrics.QueryEventListener; + +public class QueryEventListeners +{ + public static final QueryEventListener NO_OP = new BaseQueryEventListener(); + + public static final QueryEventListener.BKDIndexEventListener NO_OP_BKD_LISTENER = NO_OP.bkdIndexEventListener(); + + public static final QueryEventListener.TrieIndexEventListener NO_OP_TRIE_LISTENER = NO_OP.trieIndexEventListener(); + + public static final QueryEventListener.PostingListEventListener NO_OP_POSTINGS_LISTENER = new NoOpPostingListEventListener(); + + private static class BaseQueryEventListener implements QueryEventListener + { + @Override + public BKDIndexEventListener bkdIndexEventListener() + { + return NoOpBKDIndexEventListener.INSTANCE; + } + + @Override + public TrieIndexEventListener trieIndexEventListener() + { + return NoOpTrieIndexEventListener.INSTANCE; + } + + private enum NoOpTrieIndexEventListener implements TrieIndexEventListener + { + INSTANCE; + + @Override + public void onSegmentHit() { } + + @Override + public void onTraversalComplete(long traversalTotalTime, TimeUnit unit) { } + + @Override + public PostingListEventListener postingListEventListener() + { + return NO_OP_POSTINGS_LISTENER; + } + } + + private enum NoOpBKDIndexEventListener implements BKDIndexEventListener + { + INSTANCE; + + @Override + public void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit) { } + + @Override + public void onIntersectionEarlyExit() { } + + @Override + public void postingListsHit(int count) { } + + @Override + public void onSegmentHit() { } + + @Override + public PostingListEventListener postingListEventListener() + { + return NO_OP_POSTINGS_LISTENER; + } + } + } + + public static class NoOpPostingListEventListener implements QueryEventListener.PostingListEventListener + { + @Override + public void onAdvance() { } + + @Override + public void postingDecoded(long postingsDecoded) { } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/RAMPostingSlices.java b/src/java/org/apache/cassandra/index/sai/disk/RAMPostingSlices.java new file mode 100644 index 000000000000..e1381b0234ec --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/RAMPostingSlices.java @@ -0,0 +1,133 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; + +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.ByteBlockPool; +import org.apache.lucene.util.Counter; +import org.apache.lucene.util.mutable.MutableValueInt; + +/** + * Encodes postings as variable integers into slices + */ +class RAMPostingSlices +{ + static final int DEFAULT_TERM_DICT_SIZE = 1024; + + private final ByteBlockPool postingsPool; + private int[] postingStarts = new int[DEFAULT_TERM_DICT_SIZE]; + private int[] postingUptos = new int[DEFAULT_TERM_DICT_SIZE]; + private int[] sizes = new int[DEFAULT_TERM_DICT_SIZE]; + + RAMPostingSlices(Counter memoryUsage) + { + postingsPool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(memoryUsage)); + } + + PostingList postingList(int termID, final ByteSliceReader reader, long maxSegmentRowID) + { + initReader(reader, termID); + + final MutableValueInt lastSegmentRowId = new MutableValueInt(); + + return new PostingList() + { + @Override + public long nextPosting() throws IOException + { + if (reader.eof()) + { + return PostingList.END_OF_STREAM; + } + else + { + lastSegmentRowId.value += reader.readVInt(); + return lastSegmentRowId.value; + } + } + + @Override + public long size() + { + return sizes[termID]; + } + + @Override + public long advance(long targetRowID) + { + throw new UnsupportedOperationException(); + } + }; + } + + void initReader(ByteSliceReader reader, int termID) + { + final int upto = postingUptos[termID]; + reader.init(postingsPool, postingStarts[termID], upto); + } + + void createNewSlice(int termID) + { + if (termID >= postingStarts.length - 1) + { + postingStarts = ArrayUtil.grow(postingStarts, termID + 1); + postingUptos = ArrayUtil.grow(postingUptos, termID + 1); + sizes = ArrayUtil.grow(sizes, termID + 1); + } + + // the slice will not fit in the current block, create a new block + if ((ByteBlockPool.BYTE_BLOCK_SIZE - postingsPool.byteUpto) < ByteBlockPool.FIRST_LEVEL_SIZE) + { + postingsPool.nextBuffer(); + } + + final int upto = postingsPool.newSlice(ByteBlockPool.FIRST_LEVEL_SIZE); + postingStarts[termID] = upto + postingsPool.byteOffset; + postingUptos[termID] = upto + postingsPool.byteOffset; + } + + void writeVInt(int termID, int i) + { + while ((i & ~0x7F) != 0) + { + writeByte(termID, (byte) ((i & 0x7f) | 0x80)); + i >>>= 7; + } + writeByte(termID, (byte) i); + sizes[termID]++; + } + + private void writeByte(int termID, byte b) + { + int upto = postingUptos[termID]; + byte[] block = postingsPool.buffers[upto >> ByteBlockPool.BYTE_BLOCK_SHIFT]; + assert block != null; + int offset = upto & ByteBlockPool.BYTE_BLOCK_MASK; + if (block[offset] != 0) + { + // End of slice; allocate a new one + offset = postingsPool.allocSlice(block, offset); + block = postingsPool.buffer; + postingUptos[termID] = offset + postingsPool.byteOffset; + } + block[offset] = b; + postingUptos[termID]++; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/RAMStringIndexer.java b/src/java/org/apache/cassandra/index/sai/disk/RAMStringIndexer.java new file mode 100644 index 000000000000..84cd7187bedb --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/RAMStringIndexer.java @@ -0,0 +1,163 @@ +/* + * 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.index.sai.disk; + +import java.nio.ByteBuffer; +import java.util.NoSuchElementException; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.ByteBlockPool; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefHash; +import org.apache.lucene.util.Counter; + +/** + * Indexes strings into an on-heap inverted index to be flushed in an SSTable attached index later. + * For flushing use the PostingTerms interface. + */ +public class RAMStringIndexer +{ + private final AbstractType termComparator; + private final BytesRefHash termsHash; + private final RAMPostingSlices slices; + private final Counter bytesUsed; + + int rowCount = 0; + + private int[] lastSegmentRowID = new int[RAMPostingSlices.DEFAULT_TERM_DICT_SIZE]; + + RAMStringIndexer(AbstractType termComparator) + { + this.termComparator = termComparator; + bytesUsed = Counter.newCounter(); + + ByteBlockPool termsPool = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(bytesUsed)); + + termsHash = new BytesRefHash(termsPool); + + slices = new RAMPostingSlices(bytesUsed); + } + + long estimatedBytesUsed() + { + return bytesUsed.get(); + } + + /** + * EXPENSIVE OPERATION due to sorting the terms, only call once. + */ + // TODO: assert or throw and exception if getTermsWithPostings is called > 1 + TermsIterator getTermsWithPostings() + { + final int[] sortedTermIDs = termsHash.sort(); + + final int valueCount = termsHash.size(); + final ByteSliceReader sliceReader = new ByteSliceReader(); + + return new TermsIterator() + { + private int ordUpto = 0; + private final BytesRef br = new BytesRef(); + + @Override + public ByteBuffer getMinTerm() + { + BytesRef term = new BytesRef(); + int minTermID = sortedTermIDs[0]; + termsHash.get(minTermID, term); + return ByteBuffer.wrap(term.bytes, term.offset, term.length); + } + + @Override + public ByteBuffer getMaxTerm() + { + BytesRef term = new BytesRef(); + int maxTermID = sortedTermIDs[valueCount-1]; + termsHash.get(maxTermID, term); + return ByteBuffer.wrap(term.bytes, term.offset, term.length); + } + + public void close() {} + + @Override + public PostingList postings() + { + int termID = sortedTermIDs[ordUpto - 1]; + final int maxSegmentRowId = lastSegmentRowID[termID]; + return slices.postingList(termID, sliceReader, maxSegmentRowId); + } + + @Override + public boolean hasNext() { + return ordUpto < valueCount; + } + + @Override + public ByteComparable next() + { + if (!hasNext()) + throw new NoSuchElementException(); + + termsHash.get(sortedTermIDs[ordUpto], br); + ordUpto++; + return asByteComparable(br.bytes, br.offset, br.length); + } + + private ByteComparable asByteComparable(byte[] bytes, int offset, int length) + { + return v -> ByteSource.fixedLength(bytes, offset, length); + } + }; + } + + public long add(BytesRef term, int segmentRowId) + { + long startBytes = estimatedBytesUsed(); + int termID = termsHash.add(term); + + if (termID >= 0) + { + // firs time seeing this term, create the term's first slice ! + slices.createNewSlice(termID); + } + else + { + termID = (-termID) - 1; + } + + if (termID >= lastSegmentRowID.length - 1) + { + lastSegmentRowID = ArrayUtil.grow(lastSegmentRowID, termID + 1); + } + + int delta = segmentRowId - lastSegmentRowID[termID]; + + lastSegmentRowID[termID] = segmentRowId; + + slices.writeVInt(termID, delta); + + long allocatedBytes = estimatedBytesUsed() - startBytes; + + rowCount++; + + return allocatedBytes; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/SSTableComponentsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/SSTableComponentsWriter.java new file mode 100644 index 000000000000..d188c1270fc9 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/SSTableComponentsWriter.java @@ -0,0 +1,145 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; +import org.apache.cassandra.index.sai.disk.v1.NumericValuesWriter; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.lucene.util.IOUtils; + +/** + * Writes all SSTable-attached index token and offset structures. + */ +public class SSTableComponentsWriter +{ + protected static final Logger logger = LoggerFactory.getLogger(SSTableComponentsWriter.class); + + private final NumericValuesWriter tokenWriter; + private final NumericValuesWriter offsetWriter; + private final MetadataWriter metadataWriter; + + private final Descriptor descriptor; + private final IndexComponents indexComponents; + + private DecoratedKey currentKey; + + private long currentKeyPartitionOffset; + + public SSTableComponentsWriter(Descriptor descriptor, CompressionParams compressionParams) throws IOException + { + this.descriptor = descriptor; + + indexComponents = IndexComponents.perSSTable(descriptor, compressionParams); + this.metadataWriter = new MetadataWriter(indexComponents.createOutput(IndexComponents.GROUP_META)); + + this.tokenWriter = new NumericValuesWriter(IndexComponents.TOKEN_VALUES, + indexComponents.createOutput(IndexComponents.TOKEN_VALUES), + metadataWriter, false); + this.offsetWriter = new NumericValuesWriter(IndexComponents.OFFSETS_VALUES, + indexComponents.createOutput(IndexComponents.OFFSETS_VALUES), + metadataWriter, true); + } + + private SSTableComponentsWriter() + { + this.descriptor = null; + this.indexComponents = null; + this.metadataWriter = null; + this.tokenWriter = null; + this.offsetWriter = null; + } + + public void startPartition(DecoratedKey key, long position) + { + currentKey = key; + currentKeyPartitionOffset = position; + } + + public void nextUnfilteredCluster(Unfiltered unfiltered, long position) throws IOException + { + recordCurrentTokenOffset(); + } + + public void staticRow(Row staticRow, long position) throws IOException + { + recordCurrentTokenOffset(); + } + + private void recordCurrentTokenOffset() throws IOException + { + recordCurrentTokenOffset((long) currentKey.getToken().getTokenValue(), currentKeyPartitionOffset); + } + + @VisibleForTesting + public void recordCurrentTokenOffset(long tokenValue, long keyOffset) throws IOException + { + tokenWriter.add(tokenValue); + offsetWriter.add(keyOffset); + } + + public void complete() throws IOException + { + IOUtils.close(tokenWriter, offsetWriter, metadataWriter); + indexComponents.createGroupCompletionMarker(); + } + + public void abort(Throwable accumulator) + { + logger.debug(indexComponents.logMessage("Aborting token/offset writer for {}..."), descriptor); + IndexComponents.deletePerSSTableIndexComponents(descriptor); + } + + public static final SSTableComponentsWriter NONE = new SSTableComponentsWriter() { + + @Override + public void nextUnfilteredCluster(Unfiltered unfiltered, long position) + { + } + + @Override + public void startPartition(DecoratedKey key, long position) + { + } + + @Override + public void staticRow(Row staticRow, long position) + { + } + + @Override + public void complete() + { + } + + @Override + public void abort(Throwable accumulate) + { + } + }; +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/SSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/SSTableIndexWriter.java new file mode 100644 index 000000000000..49b918e6dae0 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/SSTableIndexWriter.java @@ -0,0 +1,358 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; +import javax.annotation.concurrent.NotThreadSafe; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.analyzer.AbstractAnalyzer; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; +import org.apache.cassandra.index.sai.utils.NamedMemoryLimiter; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.NoSpamLogger; + +/** + * Column index writer that accumulates (on-heap) indexed data from a compacted SSTable as it's being flushed to disk. + */ +@NotThreadSafe +public class SSTableIndexWriter implements ColumnIndexWriter +{ + private static final Logger logger = LoggerFactory.getLogger(SSTableIndexWriter.class); + private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES); + + public static final int MAX_STRING_TERM_SIZE = Integer.getInteger("cassandra.sai.max_string_term_size_kb", 1) * 1024; + public static final int MAX_FROZEN_TERM_SIZE = Integer.getInteger("cassandra.sai.max_frozen_term_size_kb", 5) * 1024; + public static final String TERM_OVERSIZE_MESSAGE = + "Can't add term of column {} to index for key: {}, term size {} " + + "max allowed size {}, use analyzed = true (if not yet set) for that column."; + + private final int nowInSec = FBUtilities.nowInSeconds(); + private final ColumnContext columnContext; + private final Descriptor descriptor; + private final IndexComponents indexComponents; + private final AbstractAnalyzer analyzer; + private final NamedMemoryLimiter limiter; + private final int maxTermSize; + private final BooleanSupplier isIndexValid; + + private boolean aborted = false; + + // segment writer + private SegmentBuilder currentBuilder; + private final List segments = new ArrayList<>(); + private long maxSSTableRowId; + + public SSTableIndexWriter(Descriptor descriptor, ColumnContext columnContext, NamedMemoryLimiter limiter, + BooleanSupplier isIndexValid, CompressionParams compressionParams) + { + this.columnContext = columnContext; + this.descriptor = descriptor; + this.indexComponents = IndexComponents.create(columnContext.getIndexName(), descriptor, compressionParams); + this.analyzer = columnContext.getAnalyzer(); + this.limiter = limiter; + this.isIndexValid = isIndexValid; + this.maxTermSize = columnContext.isFrozen() ? MAX_FROZEN_TERM_SIZE : MAX_STRING_TERM_SIZE; + + } + + @Override + public void addRow(DecoratedKey rowKey, long sstableRowId, Row row) throws IOException + { + if (maybeAbort()) + return; + + if (columnContext.isNonFrozenCollection()) + { + Iterator valueIterator = columnContext.getValuesOf(row, nowInSec); + if (valueIterator != null) + { + while (valueIterator.hasNext()) + { + ByteBuffer value = valueIterator.next(); + addTerm(TypeUtil.encode(value.duplicate(), columnContext.getValidator()), rowKey, sstableRowId, columnContext.getValidator()); + } + } + } + else + { + ByteBuffer value = columnContext.getValueOf(rowKey, row, nowInSec); + if (value != null) + addTerm(TypeUtil.encode(value.duplicate(), columnContext.getValidator()), rowKey, sstableRowId, columnContext.getValidator()); + } + maxSSTableRowId = sstableRowId; + } + + /** + * abort current write if index is dropped + * + * @return true if current write is aborted. + */ + private boolean maybeAbort() + { + if (aborted) + return true; + + if (isIndexValid.getAsBoolean()) + return false; + + abort(new RuntimeException(String.format("index %s is dropped", columnContext.getIndexName()))); + return true; + } + + private void addTerm(ByteBuffer term, DecoratedKey key, long sstableRowId, AbstractType type) throws IOException + { + if (term.remaining() >= maxTermSize) + { + noSpamLogger.warn(columnContext.logMessage(TERM_OVERSIZE_MESSAGE), + columnContext.getColumnName(), + columnContext.keyValidator().getString(key.getKey()), + FBUtilities.prettyPrintMemory(term.remaining()), + FBUtilities.prettyPrintMemory(maxTermSize)); + return; + } + + if (currentBuilder == null) + { + currentBuilder = newSegmentBuilder(); + } + else if (shouldFlush(sstableRowId)) + { + flushSegment(); + currentBuilder = newSegmentBuilder(); + } + + if (term.remaining() == 0) return; + + if (!TypeUtil.isLiteral(type)) + { + limiter.increment(currentBuilder.add(term, key, sstableRowId)); + } + else + { + analyzer.reset(term); + while (analyzer.hasNext()) + { + ByteBuffer token = analyzer.next(); + limiter.increment(currentBuilder.add(token, key, sstableRowId)); + } + } + } + + private boolean shouldFlush(long sstableRowId) + { + // If we've hit the minimum flush size and we've breached the global limit, flush a new segment: + boolean reachMemoryLimit = limiter.usageExceedsLimit() && currentBuilder.hasReachedMinimumFlushSize(); + + if (reachMemoryLimit) + { + logger.debug(columnContext.logMessage("Global limit of {} and minimum flush size of {} exceeded. " + + "Current builder usage is {} for {} cells. Global Usage is {}. Flushing..."), + FBUtilities.prettyPrintMemory(limiter.limitBytes()), + FBUtilities.prettyPrintMemory(currentBuilder.getMinimumFlushBytes()), + FBUtilities.prettyPrintMemory(currentBuilder.totalBytesAllocated()), + currentBuilder.getRowCount(), + FBUtilities.prettyPrintMemory(limiter.currentBytesUsed())); + } + + return reachMemoryLimit || currentBuilder.exceedsSegmentLimit(sstableRowId); + } + + private void flushSegment() throws IOException + { + long start = System.nanoTime(); + + try + { + long bytesAllocated = currentBuilder.totalBytesAllocated(); + + SegmentMetadata segmentMetadata = currentBuilder.flush(indexComponents); + + long flushMillis = Math.max(1, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); + + if (segmentMetadata != null) + { + segments.add(segmentMetadata); + + //TODO Need to look at some of these metrics + double rowCount = segmentMetadata.numRows; + if (columnContext.getIndexMetrics() != null) + columnContext.getIndexMetrics().compactionSegmentCellsPerSecond.update((long)(rowCount / flushMillis * 1000.0)); + + double segmentBytes = segmentMetadata.componentMetadatas.indexSize(); + if (columnContext.getIndexMetrics() != null) + columnContext.getIndexMetrics().compactionSegmentBytesPerSecond.update((long)(segmentBytes / flushMillis * 1000.0)); + + logger.debug(columnContext.logMessage("Flushed segment with {} cells for a total of {} to {} in {} ms."), + (long) rowCount, FBUtilities.prettyPrintMemory((long) segmentBytes), indexComponents, flushMillis); + } + + // Builder memory is released against the limiter at the conclusion of a successful + // flush. Note that any failure that occurs before this (even in term addition) will + // actuate this column writer's abort logic from the parent SSTable-level writer, and + // that abort logic will release the current builder's memory against the limiter. + long globalBytesUsed = currentBuilder.release(indexComponents); + currentBuilder = null; + logger.debug(columnContext.logMessage("Flushing index segment for SSTable {} released {}. Global segment memory usage now at {}."), + descriptor, FBUtilities.prettyPrintMemory(bytesAllocated), FBUtilities.prettyPrintMemory(globalBytesUsed)); + + } + catch (Throwable t) + { + logger.error(columnContext.logMessage("Failed to build index for SSTable {}."), descriptor, t); + indexComponents.deleteColumnIndex(); + + columnContext.getIndexMetrics().segmentFlushErrors.inc(); + + throw t; + } + } + + @Override + public void flush() throws IOException + { + if (maybeAbort()) + return; + + boolean emptySegment = currentBuilder == null || currentBuilder.isEmpty(); + logger.debug(columnContext.logMessage("Completing index flush with {}buffered data..."), emptySegment ? "no " : ""); + + try + { + // parts are present but there is something still in memory, let's flush that inline + if (!emptySegment) + { + flushSegment(); + } + + // Even an empty segment may carry some fixed memory, so remove it: + if (currentBuilder != null) + { + long bytesAllocated = currentBuilder.totalBytesAllocated(); + long globalBytesUsed = currentBuilder.release(indexComponents); + logger.debug(columnContext.logMessage("Flushing final segment for SSTable {} released {}. Global segment memory usage now at {}."), + descriptor, FBUtilities.prettyPrintMemory(bytesAllocated), FBUtilities.prettyPrintMemory(globalBytesUsed)); + } + + compactSegments(); + + writeSegmentsMetadata(); + indexComponents.createColumnCompletionMarker(); + } + finally + { + if (columnContext.getIndexMetrics() != null) + { + columnContext.getIndexMetrics().segmentsPerCompaction.update(segments.size()); + segments.clear(); + columnContext.getIndexMetrics().compactionCount.inc(); + } + } + } + + @Override + public void abort(Throwable cause) + { + aborted = true; + + logger.warn(columnContext.logMessage("Aborting SSTable index flush for {}..."), descriptor, cause); + + // It's possible for the current builder to be unassigned after we flush a final segment. + if (currentBuilder != null) + { + // If an exception is thrown out of any writer operation prior to successful segment + // flush, we will end up here, and we need to free up builder memory tracked by the limiter: + long allocated = currentBuilder.totalBytesAllocated(); + long globalBytesUsed = currentBuilder.release(indexComponents); + logger.debug(columnContext.logMessage("Aborting index writer for SSTable {} released {}. Global segment memory usage now at {}."), + descriptor, FBUtilities.prettyPrintMemory(allocated), FBUtilities.prettyPrintMemory(globalBytesUsed)); + } + + indexComponents.deleteColumnIndex(); + } + + private void compactSegments() throws IOException + { + if (segments.isEmpty()) + return; + + DecoratedKey minKey = segments.get(0).minKey; + DecoratedKey maxKey = segments.get(segments.size() - 1).maxKey; + + try (SegmentMerger segmentMerger = SegmentMerger.newSegmentMerger(columnContext.isLiteral()); + SSTableIndex.PerIndexFiles perIndexFiles = new SSTableIndex.PerIndexFiles(indexComponents, columnContext.isLiteral(), true)) + { + for (final SegmentMetadata segment : segments) + { + segmentMerger.addSegment(columnContext, segment, perIndexFiles); + } + segments.clear(); + segments.add(segmentMerger.merge(columnContext, indexComponents, minKey, maxKey, maxSSTableRowId)); + } + finally + { + indexComponents.deleteTemporaryComponents(); + } + } + + private void writeSegmentsMetadata() throws IOException + { + if (segments.isEmpty()) + return; + + try (final MetadataWriter writer = new MetadataWriter(indexComponents.createOutput(indexComponents.meta))) + { + SegmentMetadata.write(writer, segments, null); + } + catch (IOException e) + { + abort(e); + throw e; + } + } + + private SegmentBuilder newSegmentBuilder() + { + SegmentBuilder builder = TypeUtil.isLiteral(columnContext.getValidator()) + ? new SegmentBuilder.RAMStringSegmentBuilder(columnContext.getValidator(), limiter) + : new SegmentBuilder.KDTreeSegmentBuilder(columnContext.getValidator(), limiter, columnContext.getIndexWriterConfig()); + + long globalBytesUsed = limiter.increment(builder.totalBytesAllocated()); + logger.debug(columnContext.logMessage("Created new segment builder while flushing SSTable {}. Global segment memory usage now at {}."), + descriptor, FBUtilities.prettyPrintMemory(globalBytesUsed)); + + return builder; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/Segment.java b/src/java/org/apache/cassandra/index/sai/disk/Segment.java new file mode 100644 index 000000000000..ea02bb445fd3 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/Segment.java @@ -0,0 +1,185 @@ +/* + * 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.index.sai.disk; + +import java.io.Closeable; +import java.io.IOException; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; + +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.io.util.FileUtils; + +/** + * Each segment represents an on-disk index structure (kdtree/terms/postings) flushed by memory limit or token boundaries, + * or max segment rowId limit, because of lucene's limitation on 2B(Integer.MAX_VALUE). It also helps to reduce resource + * consumption for read requests as only segments that intersect with read request data range need to be loaded. + */ +public class Segment implements Closeable +{ + private final Token minKey; + private final Token.KeyBound minKeyBound; + private final Token maxKey; + private final Token.KeyBound maxKeyBound; + + // per sstable + final LongArray.Factory segmentRowIdToTokenFactory; + final LongArray.Factory segmentRowIdToOffsetFactory; + final SSTableContext.KeyFetcher keyFetcher; + // per-index + public final SSTableIndex.PerIndexFiles indexFiles; + // per-segment + public final SegmentMetadata metadata; + + private final IndexSearcher index; + private final AbstractType columnType; + + public Segment(ColumnContext columnContext, SSTableContext sstableContext, SSTableIndex.PerIndexFiles indexFiles, SegmentMetadata metadata) throws IOException + { + this.minKey = metadata.minKey.getToken(); + this.minKeyBound = minKey.minKeyBound(); + this.maxKey = metadata.maxKey.getToken(); + this.maxKeyBound = maxKey.maxKeyBound(); + + this.segmentRowIdToTokenFactory = sstableContext.tokenReaderFactory.withOffset(metadata.segmentRowIdOffset); + this.segmentRowIdToOffsetFactory = sstableContext.offsetReaderFactory.withOffset(metadata.segmentRowIdOffset); + this.keyFetcher = sstableContext.keyFetcher; + this.indexFiles = indexFiles; + this.metadata = metadata; + this.columnType = columnContext.getValidator(); + + this.index = IndexSearcher.open(columnContext.isLiteral(), this, columnContext.getColumnQueryMetrics()); + } + + @VisibleForTesting + public Segment(LongArray.Factory tokenFactory, LongArray.Factory offsetFactory, SSTableContext.KeyFetcher keyFetcher, + SSTableIndex.PerIndexFiles indexFiles, SegmentMetadata metadata, AbstractType columnType) + { + this.segmentRowIdToTokenFactory = tokenFactory; + this.segmentRowIdToOffsetFactory = offsetFactory; + this.keyFetcher = keyFetcher; + this.indexFiles = indexFiles; + this.metadata = metadata; + this.columnType = columnType; + this.minKey = null; + this.minKeyBound = null; + this.maxKey = null; + this.maxKeyBound = null; + this.index = null; + } + + @VisibleForTesting + public Segment(Token minKey, Token maxKey) + { + this.segmentRowIdToTokenFactory = null; + this.segmentRowIdToOffsetFactory = null; + this.keyFetcher = null; + this.indexFiles = null; + this.metadata = null; + this.minKey = minKey; + this.minKeyBound = minKey.minKeyBound(); + this.maxKey = maxKey; + this.maxKeyBound = maxKey.maxKeyBound(); + this.columnType = null; + this.index = null; + } + + /** + * @return true if current segment intersects with query key range + */ + public boolean intersects(AbstractBounds keyRange) + { + if (keyRange instanceof Range && ((Range)keyRange).isWrapAround()) + return keyRange.contains(minKeyBound) || keyRange.contains(maxKeyBound); + + int cmp = keyRange.right.getToken().compareTo(minKey); + // if right is minimum, it means right is the max token and bigger than maxKey. + // if right bound is less than minKey, no intersection + if (!keyRange.right.isMinimum() && (!keyRange.inclusiveRight() && cmp == 0 || cmp < 0)) + return false; + + cmp = keyRange.left.getToken().compareTo(maxKey); + // if left bound is bigger than maxKey, no intersection + if (!keyRange.isStartInclusive() && cmp == 0 || cmp > 0) + return false; + + return true; + } + + public long indexFileCacheSize() + { + return index == null ? 0 : index.indexFileCacheSize(); + } + + /** + * Search on-disk index synchronously + * + * @param expression to filter on disk index + * @param context to track per sstable cache and per query metrics + * @param defer create the iterator in a deferred state + * @return range iterator that matches given expression + */ + public RangeIterator search(Expression expression, SSTableQueryContext context, boolean defer) + { + return index.search(expression, context, defer); + } + + public AbstractType getColumnType() + { + return columnType; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Segment segment = (Segment) o; + return Objects.equal(metadata, segment.metadata); + } + + @Override + public int hashCode() + { + return Objects.hashCode(metadata); + } + + @Override + public void close() + { + FileUtils.closeQuietly(index); + } + + @Override + public String toString() + { + return String.format("Segment{metadata=%s}", metadata); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/SegmentBuilder.java b/src/java/org/apache/cassandra/index/sai/disk/SegmentBuilder.java new file mode 100644 index 000000000000..0cc030fa6e43 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/SegmentBuilder.java @@ -0,0 +1,295 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicLong; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.disk.io.BytesRefUtil; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.BKDTreeRamBuffer; +import org.apache.cassandra.index.sai.disk.v1.InvertedIndexWriter; +import org.apache.cassandra.index.sai.disk.v1.NumericIndexWriter; +import org.apache.cassandra.index.sai.utils.NamedMemoryLimiter; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; + +/** + * Creates an on-heap index data structure to be flushed to an SSTable index. + */ +@NotThreadSafe +public abstract class SegmentBuilder +{ + private static final Logger logger = LoggerFactory.getLogger(SegmentBuilder.class); + + // Served as safe net in case memory limit is not triggered or when merger merges small segments.. + public static final long LAST_VALID_SEGMENT_ROW_ID = ((long)Integer.MAX_VALUE / 2) - 1L; + private static long testLastValidSegmentRowId = -1; + + /** The number of column indexes being built globally. (Starts at one to avoid divide by zero.) */ + public static final AtomicLong ACTIVE_BUILDER_COUNT = new AtomicLong(1); + + /** Minimum flush size, dynamically updated as segment builds are started and completed/aborted. */ + private static volatile long minimumFlushBytes; + + final AbstractType termComparator; + + private final NamedMemoryLimiter limiter; + long totalBytesAllocated; + + private final long lastValidSegmentRowID; + + private boolean flushed = false; + private boolean active = true; + + // segment metadata + private long minSSTableRowId = -1; + private long maxSSTableRowId = -1; + private long segmentRowIdOffset = 0; + int rowCount = 0; + int maxSegmentRowId = -1; + // in token order + private DecoratedKey minKey, maxKey; + // in termComparator order + private ByteBuffer minTerm, maxTerm; + + public static class KDTreeSegmentBuilder extends SegmentBuilder + { + protected final byte[] buffer; + private final BKDTreeRamBuffer kdTreeRamBuffer; + private final IndexWriterConfig indexWriterConfig; + + KDTreeSegmentBuilder(AbstractType termComparator, NamedMemoryLimiter limiter, IndexWriterConfig indexWriterConfig) + { + super(termComparator, limiter); + + int typeSize = TypeUtil.fixedSizeOf(termComparator); + this.kdTreeRamBuffer = new BKDTreeRamBuffer(1, typeSize); + this.buffer = new byte[typeSize]; + this.totalBytesAllocated = this.kdTreeRamBuffer.ramBytesUsed(); + this.indexWriterConfig = indexWriterConfig; + } + + public boolean isEmpty() + { + return kdTreeRamBuffer.numRows() == 0; + } + + protected long addInternal(ByteBuffer term, int segmentRowId) + { + TypeUtil.toComparableBytes(term, termComparator, buffer); + return kdTreeRamBuffer.addPackedValue(segmentRowId, new BytesRef(buffer)); + } + + @Override + protected SegmentMetadata.ComponentMetadataMap flushInternal(IndexComponents indexComponents) throws IOException + { + try (NumericIndexWriter writer = new NumericIndexWriter(indexComponents, + TypeUtil.fixedSizeOf(termComparator), + maxSegmentRowId, + rowCount, + indexWriterConfig, + true)) + { + return writer.writeAll(kdTreeRamBuffer.asPointValues()); + } + } + } + + public static class RAMStringSegmentBuilder extends SegmentBuilder + { + final RAMStringIndexer ramIndexer; + + final BytesRefBuilder stringBuffer = new BytesRefBuilder(); + + RAMStringSegmentBuilder(AbstractType termComparator, NamedMemoryLimiter limiter) + { + super(termComparator, limiter); + + ramIndexer = new RAMStringIndexer(termComparator); + totalBytesAllocated = ramIndexer.estimatedBytesUsed(); + } + + public boolean isEmpty() + { + return ramIndexer.rowCount == 0; + } + + protected long addInternal(ByteBuffer term, int segmentRowId) + { + BytesRefUtil.copyBufferToBytesRef(term, stringBuffer); + return ramIndexer.add(stringBuffer.get(), segmentRowId); + } + + @Override + protected SegmentMetadata.ComponentMetadataMap flushInternal(IndexComponents indexComponents) throws IOException + { + try (InvertedIndexWriter writer = new InvertedIndexWriter(indexComponents, true)) + { + return writer.writeAll(ramIndexer.getTermsWithPostings()); + } + } + } + + private SegmentBuilder(AbstractType termComparator, NamedMemoryLimiter limiter) + { + this.termComparator = termComparator; + this.limiter = limiter; + this.lastValidSegmentRowID = testLastValidSegmentRowId >= 0 ? testLastValidSegmentRowId : LAST_VALID_SEGMENT_ROW_ID; + + minimumFlushBytes = limiter.limitBytes() / ACTIVE_BUILDER_COUNT.getAndIncrement(); + } + + public SegmentMetadata flush(final IndexComponents indexComponents) throws IOException + { + assert !flushed; + flushed = true; + + if (getRowCount() == 0) + { + logger.warn(indexComponents.logMessage("No rows to index during flush of SSTable {}."), indexComponents.descriptor); + return null; + } + + SegmentMetadata.ComponentMetadataMap indexMetas = flushInternal(indexComponents); + + return new SegmentMetadata(segmentRowIdOffset, rowCount, minSSTableRowId, maxSSTableRowId, minKey, maxKey, minTerm, maxTerm, indexMetas); + } + + public long add(ByteBuffer term, DecoratedKey key, long sstableRowId) + { + assert !flushed : "Cannot add to flushed segment."; + assert sstableRowId >= maxSSTableRowId; + minSSTableRowId = minSSTableRowId < 0 ? sstableRowId : minSSTableRowId; + maxSSTableRowId = sstableRowId; + + assert maxKey == null || maxKey.compareTo(key) <= 0; + minKey = minKey == null ? key : minKey; + maxKey = key; + + minTerm = TypeUtil.min(term, minTerm, termComparator); + maxTerm = TypeUtil.max(term, maxTerm, termComparator); + + if (rowCount == 0) + { + // use first global rowId in the segment as segment rowId offset + segmentRowIdOffset = sstableRowId; + } + + rowCount++; + + // segmentRowIdOffset should encode sstableRowId into Integer + int segmentRowId = castToSegmentRowId(sstableRowId, segmentRowIdOffset); + maxSegmentRowId = Math.max(maxSegmentRowId, segmentRowId); + + long bytesAllocated = addInternal(term, segmentRowId); + totalBytesAllocated += bytesAllocated; + + return bytesAllocated; + } + + public static int castToSegmentRowId(long sstableRowId, long segmentRowIdOffset) + { + int segmentRowId = Math.toIntExact(sstableRowId - segmentRowIdOffset); + + if (segmentRowId == PostingList.END_OF_STREAM) + throw new IllegalArgumentException("Illegal segment row id: END_OF_STREAM found"); + + return segmentRowId; + } + + long totalBytesAllocated() + { + return totalBytesAllocated; + } + + boolean hasReachedMinimumFlushSize() + { + return totalBytesAllocated >= minimumFlushBytes; + } + + long getMinimumFlushBytes() + { + return minimumFlushBytes; + } + + /** + * This method does three things: + * + * 1.) It decrements active builder count and updates the global minimum flush size to reflect that. + * 2.) It releases the builder's memory against its limiter. + * 3.) It defensively marks the builder inactive to make sure nothing bad happens if we try to close it twice. + * + * @param indexComponents + * + * @return the number of bytes currently used by the memory limiter + */ + long release(IndexComponents indexComponents) + { + if (active) + { + minimumFlushBytes = limiter.limitBytes() / ACTIVE_BUILDER_COUNT.decrementAndGet(); + long used = limiter.decrement(totalBytesAllocated); + active = false; + return used; + } + + logger.warn(indexComponents.logMessage("Attempted to release storage attached index segment builder memory after builder marked inactive.")); + return limiter.currentBytesUsed(); + } + + public abstract boolean isEmpty(); + + protected abstract long addInternal(ByteBuffer term, int segmentRowId); + + protected abstract SegmentMetadata.ComponentMetadataMap flushInternal(IndexComponents indexComponents) throws IOException; + + int getRowCount() + { + return rowCount; + } + + /** + * @return true if next SSTable row ID exceeds max segment row ID + */ + boolean exceedsSegmentLimit(long ssTableRowId) + { + if (getRowCount() == 0) + return false; + + // To handle the case where there are many non-indexable rows. eg. rowId-1 and rowId-3B are indexable, + // the rest are non-indexable. We should flush them as 2 separate segments, because rowId-3B is going + // to cause error in on-disk index structure with 2B limitation. + return ssTableRowId - segmentRowIdOffset > lastValidSegmentRowID; + } + + @VisibleForTesting + public static void updateLastValidSegmentRowId(long lastValidSegmentRowID) + { + testLastValidSegmentRowId = lastValidSegmentRowID; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/SegmentMerger.java b/src/java/org/apache/cassandra/index/sai/disk/SegmentMerger.java new file mode 100644 index 000000000000..2d4c486a630c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/SegmentMerger.java @@ -0,0 +1,197 @@ +/* + * 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.index.sai.disk; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.BKDReader; +import org.apache.cassandra.index.sai.disk.v1.InvertedIndexWriter; +import org.apache.cassandra.index.sai.disk.v1.NumericIndexWriter; +import org.apache.cassandra.index.sai.disk.v1.TermsReader; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.index.sai.utils.TypeUtil; + +/** + * Responsible for merging index segments into a single segment during initial index build. + */ +public interface SegmentMerger extends Closeable +{ + void addSegment(ColumnContext context, SegmentMetadata segment, SSTableIndex.PerIndexFiles indexFiles) throws IOException; + + boolean isEmpty(); + + SegmentMetadata merge(ColumnContext context, IndexComponents components, DecoratedKey minKey, DecoratedKey maxKey, long maxSSTableRowId) throws IOException; + + @SuppressWarnings("resource") + static SegmentMerger newSegmentMerger(boolean literal) + { + return literal ? new LiteralSegmentMerger() : new NumericSegmentMerger(); + } + + class LiteralSegmentMerger implements SegmentMerger + { + final List readers = new ArrayList<>(); + final List segmentTermsIterators = new ArrayList<>(); + + @Override + public void addSegment(ColumnContext context, SegmentMetadata segment, SSTableIndex.PerIndexFiles indexFiles) throws IOException + { + segmentTermsIterators.add(createTermsIterator(segment, indexFiles)); + } + + @Override + public boolean isEmpty() + { + return segmentTermsIterators.isEmpty(); + } + + @Override + public SegmentMetadata merge(ColumnContext context, IndexComponents components, DecoratedKey minKey, DecoratedKey maxKey, long maxSSTableRowId) throws IOException + { + try (final TermsIteratorMerger merger = new TermsIteratorMerger(segmentTermsIterators.toArray(new TermsIterator[0]), context.getValidator())) + { + + SegmentMetadata.ComponentMetadataMap indexMetas; + long numRows; + + try (InvertedIndexWriter indexWriter = new InvertedIndexWriter(components, false)) + { + indexMetas = indexWriter.writeAll(merger); + numRows = indexWriter.getPostingsCount(); + } + return new SegmentMetadata(0, + numRows, + merger.minSSTableRowId, + merger.maxSSTableRowId, + minKey, + maxKey, + merger.getMinTerm(), + merger.getMaxTerm(), + indexMetas); + } + } + + @Override + public void close() throws IOException + { + readers.forEach(TermsReader::close); + } + + @SuppressWarnings("resource") + private TermsIterator createTermsIterator(SegmentMetadata segment, SSTableIndex.PerIndexFiles indexFiles) throws IOException + { + final long root = segment.getIndexRoot(indexFiles.components().termsData); + assert root >= 0; + + final Map map = segment.componentMetadatas.get(IndexComponents.NDIType.TERMS_DATA).attributes; + final String footerPointerString = map.get(SAICodecUtils.FOOTER_POINTER); + final long footerPointer = footerPointerString == null ? -1 : Long.parseLong(footerPointerString); + + final TermsReader termsReader = new TermsReader(indexFiles.components(), + indexFiles.termsData().sharedCopy(), + indexFiles.postingLists().sharedCopy(), + root, + footerPointer); + readers.add(termsReader); + return termsReader.allTerms(segment.segmentRowIdOffset, QueryEventListeners.NO_OP_TRIE_LISTENER); + } + } + + class NumericSegmentMerger implements SegmentMerger + { + final List segmentIterators = new ArrayList<>(); + final List readers = new ArrayList<>(); + + ByteBuffer minTerm = null, maxTerm = null; + + @Override + public void addSegment(ColumnContext context, SegmentMetadata segment, SSTableIndex.PerIndexFiles indexFiles) throws IOException + { + minTerm = TypeUtil.min(segment.minTerm, minTerm, context.getValidator()); + maxTerm = TypeUtil.max(segment.maxTerm, maxTerm, context.getValidator()); + + segmentIterators.add(createIteratorState(segment, indexFiles)); + } + + @Override + public boolean isEmpty() + { + return segmentIterators.isEmpty(); + } + + @Override + public SegmentMetadata merge(ColumnContext context, IndexComponents components, DecoratedKey minKey, DecoratedKey maxKey, long maxSSTableRowId) throws IOException + { + final MergeOneDimPointValues merger = new MergeOneDimPointValues(segmentIterators, context.getValidator()); + + final SegmentMetadata.ComponentMetadataMap componentMetadataMap; + try (NumericIndexWriter indexWriter = new NumericIndexWriter(components, + TypeUtil.fixedSizeOf(context.getValidator()), + maxSSTableRowId, + Integer.MAX_VALUE, + context.getIndexWriterConfig(), + false)) + { + componentMetadataMap = indexWriter.writeAll(merger); + } + return new SegmentMetadata(0, + merger.getNumRows(), + merger.getMinRowID(), + merger.getMaxRowID(), + minKey, + maxKey, + minTerm, + maxTerm, + componentMetadataMap); + } + + @Override + public void close() throws IOException + { + segmentIterators.forEach(BKDReader.IteratorState::close); + readers.forEach(BKDReader::close); + } + + @SuppressWarnings("resource") + private BKDReader.IteratorState createIteratorState(SegmentMetadata segment, SSTableIndex.PerIndexFiles indexFiles) throws IOException + { + final long bkdPosition = segment.getIndexRoot(indexFiles.components().kdTree); + assert bkdPosition >= 0; + final long postingsPosition = segment.getIndexRoot(indexFiles.components().kdTreePostingLists); + assert postingsPosition >= 0; + + final BKDReader bkdReader = new BKDReader(indexFiles.components(), + indexFiles.kdtree().sharedCopy(), + bkdPosition, + indexFiles.kdtreePostingLists().sharedCopy(), + postingsPosition); + readers.add(bkdReader); + return bkdReader.iteratorState(rowid -> rowid + segment.segmentRowIdOffset); + } + } +} + diff --git a/src/java/org/apache/cassandra/index/sai/disk/SegmentMetadata.java b/src/java/org/apache/cassandra/index/sai/disk/SegmentMetadata.java new file mode 100644 index 000000000000..543932b90a03 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/SegmentMetadata.java @@ -0,0 +1,406 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.disk.io.CryptoUtils; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.io.RAMIndexOutput; +import org.apache.cassandra.index.sai.disk.v1.MetadataSource; +import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; +import org.apache.cassandra.io.compress.ICompressor; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; + +/** + * Multiple {@link SegmentMetadata} are stored in {@link IndexComponents.NDIType#meta} file, each corresponds to an on-disk + * index segment. + */ +public class SegmentMetadata implements Comparable +{ + private static final String NAME = "SegmentMetadata"; + + /** + * Used to retrieve sstableRowId which equals to offset plus segmentRowId. + */ + public final long segmentRowIdOffset; + + /** + * Min and max sstable rowId in current segment. + * + * For index generated by compaction, minSSTableRowId is the same as segmentRowIdOffset. + * But for flush, segmentRowIdOffset is taken from previous segment's maxSSTableRowId. + */ + public final long minSSTableRowId; + public final long maxSSTableRowId; + + /** + * number of indexed rows (aka. pair of term and segmentRowId) in current segment + */ + public final long numRows; + + /** + * Ordered by their token position in current segment + */ + public final DecoratedKey minKey; + public final DecoratedKey maxKey; + + /** + * Minimum and maximum indexed column value ordered by its {@link org.apache.cassandra.db.marshal.AbstractType}. + */ + public final ByteBuffer minTerm, maxTerm; + + /** + * Root, offset, length for each index structure in the segment. + * + * Note: postings block offsets are stored in terms dictionary, no need to worry about its root. + */ + public final ComponentMetadataMap componentMetadatas; + + SegmentMetadata(long segmentRowIdOffset, + long numRows, + long minSSTableRowId, + long maxSSTableRowId, + DecoratedKey minKey, + DecoratedKey maxKey, + ByteBuffer minTerm, + ByteBuffer maxTerm, + ComponentMetadataMap componentMetadatas) + { + assert numRows < Integer.MAX_VALUE; + Objects.requireNonNull(minKey); + Objects.requireNonNull(maxKey); + Objects.requireNonNull(minTerm); + Objects.requireNonNull(maxTerm); + + this.segmentRowIdOffset = segmentRowIdOffset; + this.minSSTableRowId = minSSTableRowId; + this.maxSSTableRowId = maxSSTableRowId; + this.numRows = numRows; + this.minKey = minKey; + this.maxKey = maxKey; + this.minTerm = minTerm; + this.maxTerm = maxTerm; + this.componentMetadatas = componentMetadatas; + } + + private static final Logger logger = LoggerFactory.getLogger(SegmentMetadata.class); + + @SuppressWarnings("resource") + private SegmentMetadata(IndexInput input, ICompressor compressor) throws IOException + { + this.segmentRowIdOffset = input.readLong(); + + numRows = input.readLong(); + minSSTableRowId = input.readLong(); + maxSSTableRowId = input.readLong(); + minKey = DatabaseDescriptor.getPartitioner().decorateKey(readBytes(input)); + maxKey = DatabaseDescriptor.getPartitioner().decorateKey(readBytes(input)); + + if (compressor != null) + { + IndexInput cryptoIn = CryptoUtils.uncompress(input, compressor); + + assert cryptoIn.length() > 0; + + minTerm = readBytes(cryptoIn); + maxTerm = readBytes(cryptoIn); + } + else + { + minTerm = readBytes(input); + maxTerm = readBytes(input); + } + componentMetadatas = new ComponentMetadataMap(input); + } + + @SuppressWarnings("resource") + public static List load(MetadataSource source, ICompressor compressor) throws IOException + { + IndexInput input = source.get(NAME); + + int segmentCount = input.readVInt(); + + List segmentMetadata = new ArrayList<>(segmentCount); + + for (int i = 0; i < segmentCount; i++) + { + segmentMetadata.add(new SegmentMetadata(input, compressor)); + } + + return segmentMetadata; + } + + /** + * Writes disk metadata for the given segment list. + */ + @SuppressWarnings("resource") + public static void write(MetadataWriter writer, List segments, ICompressor compressor) throws IOException + { + try (IndexOutput output = writer.builder(NAME)) + { + output.writeVInt(segments.size()); + + for (SegmentMetadata metadata : segments) + { + output.writeLong(metadata.segmentRowIdOffset); + output.writeLong(metadata.numRows); + output.writeLong(metadata.minSSTableRowId); + output.writeLong(metadata.maxSSTableRowId); + + if (compressor != null) + { + Stream.of(metadata.minKey.getKey(), metadata.maxKey.getKey()).forEach(bb -> writeBytes(bb, output)); + + RAMIndexOutput out = new RAMIndexOutput(""); + writeBytes(metadata.minTerm, out); + writeBytes(metadata.maxTerm, out); + + CryptoUtils.compress(new BytesRef(out.getBytes(), 0, (int)out.getFilePointer()), output, compressor); + } + else + { + Stream.of(metadata.minKey.getKey(), metadata.maxKey.getKey(), metadata.minTerm, metadata.maxTerm).forEach(bb -> writeBytes(bb, output)); + } + + metadata.componentMetadatas.write(output); + } + } + } + + @Override + public int compareTo(SegmentMetadata other) + { + return Long.compare(this.segmentRowIdOffset, other.segmentRowIdOffset); + } + + @Override + public String toString() + { + return "SegmentMetadata{" + + "segmentRowIdOffset=" + segmentRowIdOffset + + ", minSSTableRowId=" + minSSTableRowId + + ", maxSSTableRowId=" + maxSSTableRowId + + ", numRows=" + numRows + + ", componentMetadatas=" + componentMetadatas + + '}'; + } + + private static ByteBuffer readBytes(IndexInput input) throws IOException + { + int len = input.readVInt(); + byte[] bytes = new byte[len]; + input.readBytes(bytes, 0, len); + return ByteBuffer.wrap(bytes); + } + + private static void writeBytes(ByteBuffer buf, IndexOutput out) + { + try + { + byte[] bytes = ByteBufferUtil.getArray(buf); + out.writeVInt(bytes.length); + out.writeBytes(bytes, 0, bytes.length); + } + catch (IOException ioe) + { + throw new RuntimeException(ioe); + } + } + + long getIndexRoot(IndexComponents.IndexComponent component) + { + return componentMetadatas.get(component.ndiType).root; + } + + public long getIndexOffset(IndexComponents.IndexComponent component) + { + return componentMetadatas.get(component.ndiType).offset; + } + + public long getIndexLength(IndexComponents.IndexComponent component) + { + return componentMetadatas.get(component.ndiType).length; + } + + public static class ComponentMetadataMap + { + private final Map metas = new HashMap<>(); + + ComponentMetadataMap(IndexInput input) throws IOException + { + int size = input.readInt(); + + for (int i = 0; i < size; i++) + { + metas.put(IndexComponents.NDIType.valueOf(input.readString()), new ComponentMetadata(input)); + } + } + + public ComponentMetadataMap() + { + } + + public void put(IndexComponents.NDIType ndiType, long root, long offset, long length) + { + metas.put(ndiType, new ComponentMetadata(root, offset, length)); + } + + public void put(IndexComponents.NDIType ndiType, long root, long offset, long length, Map additionalMap) + { + metas.put(ndiType, new ComponentMetadata(root, offset, length, additionalMap)); + } + + private void write(IndexOutput output) throws IOException + { + output.writeInt(metas.size()); + + for (Map.Entry entry : metas.entrySet()) + { + output.writeString(entry.getKey().name()); + entry.getValue().write(output); + } + } + + public ComponentMetadata get(IndexComponents.NDIType ndiType) + { + if (!metas.containsKey(ndiType)) + throw new IllegalArgumentException(ndiType + " ComponentMetadata not found"); + + return metas.get(ndiType); + } + + public Map> asMap() + { + Map> metaAttributes = new HashMap<>(); + + for (Map.Entry entry : metas.entrySet()) + { + String name = entry.getKey().name; + ComponentMetadata metadata = entry.getValue(); + + Map componentAttributes = metadata.asMap(); + + assert !metaAttributes.containsKey(name) : "Found duplicate index type: " + name; + metaAttributes.put(name, componentAttributes); + } + + return metaAttributes; + } + + @Override + public String toString() + { + return "ComponentMetadataMap{" + + "metas=" + metas + + '}'; + } + + public double indexSize() + { + return metas.values().stream().mapToLong(meta -> meta.length).sum(); + } + } + + public static class ComponentMetadata + { + public static final String ROOT = "Root"; + public static final String OFFSET = "Offset"; + public static final String LENGTH = "Length"; + + public final long root; + public final long offset; + public final long length; + public final Map attributes; + + ComponentMetadata(long root, long offset, long length) + { + this.root = root; + this.offset = offset; + this.length = length; + this.attributes = Collections.emptyMap(); + } + + ComponentMetadata(long root, long offset, long length, Map attributes) + { + this.root = root; + this.offset = offset; + this.length = length; + this.attributes = attributes; + } + + ComponentMetadata(IndexInput input) throws IOException + { + this.root = input.readLong(); + this.offset = input.readLong(); + this.length = input.readLong(); + int size = input.readInt(); + + attributes = new HashMap<>(size); + for (int x=0; x < size; x++) + { + String key = input.readString(); + String value = input.readString(); + + attributes.put(key, value); + } + } + + public void write(IndexOutput output) throws IOException + { + output.writeLong(root); + output.writeLong(offset); + output.writeLong(length); + + output.writeInt(attributes.size()); + for (Map.Entry entry : attributes.entrySet()) + { + output.writeString(entry.getKey()); + output.writeString(entry.getValue()); + } + } + + @Override + public String toString() + { + return String.format("ComponentMetadata{root=%d, offset=%d, length=%d, attributes=%s}", root, offset, length, attributes.toString()); + } + + public Map asMap() + { + return ImmutableMap.builder().putAll(attributes).put(OFFSET, Long.toString(offset)).put(LENGTH, Long.toString(length)).put(ROOT, Long.toString(root)).build(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java new file mode 100644 index 000000000000..66cae625f792 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java @@ -0,0 +1,255 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.Collection; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import com.google.common.base.Stopwatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.index.sai.memory.RowMapping; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; +import org.apache.cassandra.schema.CompressionParams; + +/** + * Writes all on-disk index structures attached to a given SSTable. + */ +public class StorageAttachedIndexWriter implements SSTableFlushObserver +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final Descriptor descriptor; + private final Collection indices; + private final Collection columnIndexWriters; + private final SSTableComponentsWriter sstableComponentsWriter; + private final Stopwatch stopwatch = Stopwatch.createUnstarted(); + private final RowMapping rowMapping; + + private DecoratedKey currentKey; + private boolean tokenOffsetWriterCompleted = false; + private boolean aborted = false; + + private long sstableRowId = 0; + + public StorageAttachedIndexWriter(Descriptor descriptor, + Collection indices, + LifecycleNewTracker tracker, CompressionParams compressionParams) throws IOException + { + this(descriptor, indices, tracker, false, compressionParams); + } + + public StorageAttachedIndexWriter(Descriptor descriptor, + Collection indices, + LifecycleNewTracker tracker, + boolean perColumnOnly, CompressionParams compressionParams) throws IOException + { + this.descriptor = descriptor; + this.indices = indices; + this.rowMapping = RowMapping.create(tracker.opType()); + this.columnIndexWriters = indices.stream().map(i -> i.newIndexWriter(descriptor, tracker, rowMapping, compressionParams)) + .filter(Objects::nonNull) // a null here means the column had no data to flush + .collect(Collectors.toList()); + + this.sstableComponentsWriter = perColumnOnly ? SSTableComponentsWriter.NONE : new SSTableComponentsWriter(descriptor, compressionParams); + } + + @Override + public void begin() + { + logger.debug(logMessage("Starting partition iteration for storage attached index flush for SSTable {}..."), descriptor); + stopwatch.start(); + } + + @Override + public void startPartition(DecoratedKey key, long position) + { + if (aborted) return; + + currentKey = key; + sstableComponentsWriter.startPartition(key, position); + } + + @Override + public void nextUnfilteredCluster(Unfiltered unfiltered, long position) + { + if (aborted) return; + + try + { + // Ignore range tombstones... + if (unfiltered.isRow()) + { + sstableComponentsWriter.nextUnfilteredCluster(unfiltered, position); + rowMapping.add(currentKey, unfiltered, sstableRowId); + + for (ColumnIndexWriter w : columnIndexWriters) + { + w.addRow(currentKey, sstableRowId, (Row) unfiltered); + } + + sstableRowId++; + } + } + catch (Throwable t) + { + logger.error(logMessage("Failed to record a row during an index build"), t); + abort(t, true); + } + } + + @Override + public void partitionLevelDeletion(DeletionTime deletionTime, long position) + { + // Deletions (including partition deletions) are accounted for during reads. + } + + @Override + public void staticRow(Row staticRow, long position) + { + if (aborted) return; + + if (staticRow.isEmpty()) + return; + + try + { + sstableComponentsWriter.staticRow(staticRow, position); + rowMapping.add(currentKey, staticRow, sstableRowId); + + for (ColumnIndexWriter w : columnIndexWriters) + { + w.addRow(currentKey, sstableRowId, staticRow); + } + + sstableRowId++; + } + catch (Throwable t) + { + logger.error(logMessage("Failed to record a static row during an index build"), t); + abort(t, true); + } + } + + @Override + public void complete() + { + if (aborted) return; + + logger.debug(logMessage("Completed partition iteration for index flush for SSTable {}. Elapsed time: {} ms"), + descriptor, stopwatch.elapsed(TimeUnit.MILLISECONDS)); + + try + { + sstableComponentsWriter.complete(); + tokenOffsetWriterCompleted = true; + + logger.debug(logMessage("Flushed tokens and offsets for SSTable {}. Elapsed time: {} ms."), + descriptor, stopwatch.elapsed(TimeUnit.MILLISECONDS)); + + rowMapping.complete(); + + for (ColumnIndexWriter columnIndexWriter : columnIndexWriters) + { + columnIndexWriter.flush(); + } + } + catch (Throwable t) + { + logger.error(logMessage("Failed to complete an index build"), t); + abort(t, true); + } + } + + /** + * Aborts all column index writers and, only if they have not yet completed, SSTable-level component writers. + * + * @param accumulator the initial exception thrown from the failed writer + */ + @Override + public void abort(Throwable accumulator) + { + abort(accumulator, false); + } + + /** + * + * @param accumulator original cause of the abort + * @param fromIndex true if the cause of the abort was the index itself, false otherwise + */ + public void abort(Throwable accumulator, boolean fromIndex) + { + // Mark the write aborted, so we can short-circuit any further operations on the component writers. + aborted = true; + + // Make any indexes involved in this transaction non-queryable, as they will likely not match the backing table. + if (fromIndex) + indices.forEach(StorageAttachedIndex::makeIndexNonQueryable); + + for (ColumnIndexWriter writer : columnIndexWriters) + { + try + { + writer.abort(accumulator); + } + catch (Throwable t) + { + if (accumulator != null) + { + accumulator.addSuppressed(t); + } + } + } + + if (!tokenOffsetWriterCompleted) + { + // If the token/offset files have already been written successfully, they can be reused later. + sstableComponentsWriter.abort(accumulator); + } + } + + /** + * A helper method for constructing consistent log messages. This method is different to similar helper + * methods in that log messages generated in this class are not necessarily related to a single index + * so the log message is decorated as follows: + * + * [ks.tb.*] Log message + * + * @param message The raw content of a logging message. + * + * @return A log message with the proper keyspace and table name prepended to it. + */ + public String logMessage(String message) + { + // Index names are unique only within a keyspace. + return String.format("[%s.%s.*] %s", descriptor.ksname, descriptor.cfname, message); + } + +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/TermsIterator.java b/src/java/org/apache/cassandra/index/sai/disk/TermsIterator.java new file mode 100644 index 000000000000..368f8e30a64c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/TermsIterator.java @@ -0,0 +1,45 @@ +/* + * 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.index.sai.disk; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/** + * Iterator to step through terms to obtain {@link PostingList} for the current term. + * + * Term enumerations are always ordered by their {@link ByteSource}. + */ +@NotThreadSafe +public interface TermsIterator extends Iterator, Closeable +{ + /** + * Get {@link PostingList} for the current term. + */ + PostingList postings() throws IOException; + + ByteBuffer getMinTerm(); + + ByteBuffer getMaxTerm(); +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/TermsIteratorMerger.java b/src/java/org/apache/cassandra/index/sai/disk/TermsIteratorMerger.java new file mode 100644 index 000000000000..bfdad083ba56 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/TermsIteratorMerger.java @@ -0,0 +1,141 @@ +/* + * 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.index.sai.disk; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.PriorityQueue; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.disk.v1.MergePostingList; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; + +public class TermsIteratorMerger implements TermsIterator +{ + private final TermsIterator[] iterators; + private final MergingIterator mergedIterator; + private final AbstractType type; + + public long maxSSTableRowId = -1; + public long minSSTableRowId = Long.MAX_VALUE; + public ByteComparable minTerm; + public ByteComparable maxTerm; + + public TermsIteratorMerger(final TermsIterator[] iterators, AbstractType type) + { + this.iterators = iterators; + this.mergedIterator = new MergingIterator(type, iterators); + this.type = type; + } + + @Override + public ByteBuffer getMinTerm() + { + byte[] bytes = ByteSourceInverse.readBytes(minTerm.asPeekableBytes(ByteComparable.Version.OSS41)); + return ByteBuffer.wrap(bytes); + } + + @Override + public ByteBuffer getMaxTerm() + { + byte[] bytes = ByteSourceInverse.readBytes(maxTerm.asPeekableBytes(ByteComparable.Version.OSS41)); + return ByteBuffer.wrap(bytes); + } + + @Override + public void close() throws IOException + { + for (TermsIterator iterator : iterators) + iterator.close(); + } + + @Override + public boolean hasNext() + { + return mergedIterator.hasNext(); + } + + @Override + @SuppressWarnings("resource") + public PostingList postings() throws IOException + { + final PriorityQueue postingLists = new PriorityQueue<>(100, Comparator.comparingLong(PostingList.PeekablePostingList::peek)); + for (int x = 0; x < mergedIterator.getNumTop(); x++) + { + final int index = mergedIterator.top[x].index; + final TermsIterator termsIterator = iterators[index]; + final PostingList postings = termsIterator.postings(); + + postingLists.add(postings.peekable()); + } + return new MonitoringPostingList(MergePostingList.merge(postingLists)); + } + + @Override + public ByteComparable next() + { + ByteComparable nextTerm = mergedIterator.next(); + minTerm = type.isReversed() ? TypeUtil.max(nextTerm, minTerm) : TypeUtil.min(nextTerm, minTerm); + maxTerm = type.isReversed() ? TypeUtil.min(nextTerm, maxTerm) : TypeUtil.max(nextTerm, minTerm); + + return nextTerm; + } + + private class MonitoringPostingList implements PostingList + { + private final PostingList monitored; + + private MonitoringPostingList(PostingList monitored) + { + this.monitored = monitored; + } + + @Override + public long nextPosting() throws IOException + { + long next = monitored.nextPosting(); + if (next != PostingList.END_OF_STREAM) + { + minSSTableRowId = Math.min(minSSTableRowId, next); + maxSSTableRowId = Math.max(maxSSTableRowId, next); + } + return next; + } + + @Override + public long size() + { + return monitored.size(); + } + + @Override + public long advance(long targetRowID) throws IOException + { + return monitored.advance(targetRowID); + } + + @Override + public void close() throws IOException + { + monitored.close(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/Version.java b/src/java/org/apache/cassandra/index/sai/disk/format/Version.java new file mode 100644 index 000000000000..85b6d2ce0b7b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/format/Version.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.index.sai.disk.format; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * Format version of indexing component, denoted as [major][minor]. Same forward-compatibility rules apply as to + * {@link org.apache.cassandra.io.sstable.format.Version}. + */ +public class Version +{ + private static final Version AA = new Version('a', 'a'); + + public static final Version EARLIEST = AA; + public static final Version LATEST = AA; + + private final String version; + + public Version(char major, char minor) + { + this.version = major + "" + minor; + } + + public static Version parse(String input) + { + checkArgument(input.length() == 2); + return new Version(input.charAt(0), input.charAt(1)); + } + + @Override + public String toString() + { + return version; + } + + public boolean onOrAfter(Version other) + { + return version.compareTo(other.version) >= 0; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/BytesRefUtil.java b/src/java/org/apache/cassandra/index/sai/disk/io/BytesRefUtil.java new file mode 100644 index 000000000000..ccad8f85f51a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/BytesRefUtil.java @@ -0,0 +1,38 @@ +/* + * 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.index.sai.disk.io; + + +import java.nio.ByteBuffer; + +import org.apache.cassandra.utils.FastByteOperations; +import org.apache.lucene.util.BytesRefBuilder; + +public final class BytesRefUtil +{ + private BytesRefUtil() {} + + public static void copyBufferToBytesRef(ByteBuffer buffer, BytesRefBuilder stringBuffer) + { + int length = buffer.remaining(); + stringBuffer.clear(); + stringBuffer.grow(length); + FastByteOperations.copy(buffer, buffer.position(), stringBuffer.bytes(), 0, buffer.remaining()); + stringBuffer.setLength(length); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/CryptoUtils.java b/src/java/org/apache/cassandra/index/sai/disk/io/CryptoUtils.java new file mode 100644 index 000000000000..06b3572edfd1 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/CryptoUtils.java @@ -0,0 +1,134 @@ +/* + * 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.index.sai.disk.io; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.cassandra.io.compress.CompressionMetadata; +import org.apache.cassandra.io.compress.ICompressor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.lucene.store.ByteArrayIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; + +public class CryptoUtils +{ + + public static CompressionMetadata getCompressionMeta(SSTableReader ssTableReader) + { + return ssTableReader.compression ? ssTableReader.getCompressionMetadata() : null; + } + + public static CompressionParams getCompressionParams(SSTableReader ssTableReader) + { + return getCompressionParams(getCompressionMeta(ssTableReader)); + } + + public static CompressionParams getCompressionParams(CompressionMetadata meta) + { + return meta != null ? meta.parameters : null; + } + + //TODO Encryption tidyup +// public static ICompressor getEncryptionCompressor(CompressionParams compressionParams) +// { +// ICompressor compressor = compressionParams != null ? compressionParams.getSstableCompressor() : null; +// return compressor != null ? compressor.encryptionOnly() : null; +// } +// +// public static boolean isCryptoEnabled(CompressionParams params) +// { +// ICompressor sstableCompressor = params != null ? params.getSstableCompressor() : null; +// return sstableCompressor != null && sstableCompressor.encryptionOnly() != null ? true : false; +// } + + public static IndexInput uncompress(IndexInput input, ICompressor compressor) throws IOException + { + return uncompress(input, compressor, + new BytesRef(new byte[16]), new BytesRef(new byte[16]) + ); + } + + /** + * Takes an {@link IndexInput} with compressed/encrypted data and returns another {@link IndexInput} with + * that data uncompressed/decrypted. + */ + public static IndexInput uncompress(IndexInput input, ICompressor compressor, BytesRef compBytes, BytesRef uncompBytes) throws IOException + { + final int uncompBytesLen = input.readVInt(); + final int compBytesLength = input.readVInt(); + + assert compBytesLength > 0 : "uncompBytesLen="+uncompBytesLen+" compBytesLength="+compBytesLength; + + compBytes.bytes = ArrayUtil.grow(compBytes.bytes, compBytesLength); + + input.readBytes(compBytes.bytes, 0, compBytesLength); + + if (uncompBytes.bytes == BytesRef.EMPTY_BYTES) + { + // if EMPTY_BYTES use an exact new byte array + uncompBytes.bytes = new byte[uncompBytesLen]; + uncompBytes.length = uncompBytesLen; + } + else + { + uncompBytes.bytes = ArrayUtil.grow(uncompBytes.bytes, uncompBytesLen); + uncompBytes.length = uncompBytesLen; + } + compressor.uncompress(compBytes.bytes, 0, compBytesLength, uncompBytes.bytes, 0); + + return new ByteArrayIndexInput("", uncompBytes.bytes, 0, uncompBytesLen); + } + + public static void compress(BytesRef uncompBytes, + IndexOutput out, ICompressor compressor) throws IOException + { + compress(uncompBytes, new BytesRef(new byte[16]), out, compressor); + } + + public static void compress(BytesRef uncompBytes, BytesRef compBytes, + IndexOutput out, ICompressor compressor) throws IOException + { + ByteBuffer input = ByteBuffer.wrap(uncompBytes.bytes, 0, uncompBytes.length); + + final int initCompLen = compressor.initialCompressedBufferLength(uncompBytes.length); + + compBytes.bytes = ArrayUtil.grow(compBytes.bytes, initCompLen); + compBytes.length = initCompLen; + + ByteBuffer output = ByteBuffer.wrap(compBytes.bytes); + + compressor.compress(input, output); + + final int compLen = output.position(); + + compBytes.length = compLen; + + assert uncompBytes.length > 0; + assert compLen > 0; + + out.writeVInt(uncompBytes.length); + out.writeVInt(compLen); + + out.writeBytes(compBytes.bytes, compLen); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/EmptyDirectory.java b/src/java/org/apache/cassandra/index/sai/disk/io/EmptyDirectory.java new file mode 100644 index 000000000000..8269304f4ed2 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/EmptyDirectory.java @@ -0,0 +1,100 @@ +/* + * 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.index.sai.disk.io; + +import java.util.Collection; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.Lock; + +/** + * Always empty directory. Any operations to create, delete or open index files are unsupported. + */ +public final class EmptyDirectory extends Directory +{ + public static final Directory INSTANCE = new EmptyDirectory(); + + @Override + public String[] listAll() + { + return new String[0]; + } + + @Override + public void close() + { + // no-op + } + + @Override + public void deleteFile(String name) + { + throw new UnsupportedOperationException(); + } + + @Override + public long fileLength(String name) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexOutput createOutput(String name, IOContext context) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) + { + throw new UnsupportedOperationException(); + } + + @Override + public void sync(Collection names) + { + throw new UnsupportedOperationException(); + } + + @Override + public void syncMetaData() + { + throw new UnsupportedOperationException(); + } + + @Override + public void rename(String source, String dest) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexInput openInput(String name, IOContext context) + { + throw new UnsupportedOperationException(); + } + + @Override + public Lock obtainLock(String name) + { + throw new UnsupportedOperationException(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/FilterIndexInput.java b/src/java/org/apache/cassandra/index/sai/disk/io/FilterIndexInput.java new file mode 100644 index 000000000000..5a8092d8222b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/FilterIndexInput.java @@ -0,0 +1,86 @@ +/* + * 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.index.sai.disk.io; + +import java.io.IOException; + +import org.apache.lucene.store.IndexInput; + +public abstract class FilterIndexInput extends IndexInput +{ + private final IndexInput delegate; + + protected FilterIndexInput(IndexInput delegate) + { + super(delegate.toString()); + this.delegate = delegate; + } + + public IndexInput getDelegate() + { + return delegate; + } + + @Override + public void close() throws IOException + { + delegate.close(); + } + + @Override + public long getFilePointer() + { + return delegate.getFilePointer(); + } + + @Override + public void seek(long pos) throws IOException + { + delegate.seek(pos); + } + + @Override + public long length() + { + return delegate.length(); + } + + @Override + public IndexInput slice(String sliceDescription, long offset, long length) throws IOException + { + return delegate.slice(sliceDescription, offset, length); + } + + @Override + public byte readByte() throws IOException + { + return delegate.readByte(); + } + + @Override + public void readBytes(byte[] b, int offset, int len) throws IOException + { + delegate.readBytes(b, offset, len); + } + + @Override + public String toString() + { + return delegate.toString(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/IndexComponents.java b/src/java/org/apache/cassandra/index/sai/disk/io/IndexComponents.java new file mode 100644 index 000000000000..20e48d0a9c03 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/IndexComponents.java @@ -0,0 +1,768 @@ +/* + * 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.index.sai.disk.io; + +import java.io.File; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Stream; +import java.util.zip.CRC32; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; +import com.google.common.base.Strings; +import com.google.common.collect.ObjectArrays; +import com.google.common.io.Files; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.index.sai.disk.SegmentMetadata; +import org.apache.cassandra.index.sai.disk.v1.MetadataSource; +import org.apache.cassandra.index.sai.disk.v1.NumericValuesMeta; +import org.apache.cassandra.index.sai.disk.v1.PostingsWriter; +import org.apache.cassandra.index.sai.disk.v1.TrieTermsDictionaryWriter; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.io.util.SequentialWriter; +import org.apache.cassandra.io.util.SequentialWriterOption; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.bkd.BKDWriter; + +/** + * //TODO Need to consider how we handle encryption in OS + * The {@link Component}s that storage-attached indexing attaches to an SSTable. + * + * It allows us to unify index file creation, and ensures they will follow the same naming convention. + */ +public class IndexComponents +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + public static final String TYPE_PREFIX = "SAI"; + private static final String PER_SSTABLE_FILE_NAME_FORMAT = TYPE_PREFIX + "_%s.db"; +// public static final String LEGACY_PER_COLUMN_FILE_NAME_FORMAT = "%s_" + PER_SSTABLE_FILE_NAME_FORMAT; + public static final String PER_COLUMN_FILE_NAME_FORMAT = TYPE_PREFIX + "_%s_%s.db"; + + public static class IndexComponent extends Component + { + public final NDIType ndiType; + + private IndexComponent(NDIType ndiType, String name) + { + super(Type.CUSTOM, name); + this.ndiType = ndiType; + } + } + + public enum NDIType + { + // per-column components + /** + * Stores multiple {@link SegmentMetadata}s. + */ + META("Meta", false), + /** + * KDTree written by {@link BKDWriter} indexes mappings of term to one ore more segment row IDs + * (segment row ID = SSTable row ID - segment row ID offset). + */ + KD_TREE("KDTree", false), + KD_TREE_POSTING_LISTS("KDTreePostingLists", false), + /** + * Term dictionary written by {@link TrieTermsDictionaryWriter} stores mappings of term and + * file pointer to posting block on posting file. + */ + TERMS_DATA("TermsData", false, false, true), + /** + * Stores postings written by {@link PostingsWriter} + */ + POSTING_LISTS("PostingLists", false), + /** + * If present indicates that the column index build completed successfully + */ + COLUMN_COMPLETION_MARKER("ColumnComplete", false, true), + + // per-sstable components + /** + * Partition key token value for rows including row tombstone and static row. (access key is rowId) + */ + TOKEN_VALUES("TokenValues"), + /** + * Partition key offset in sstable data file for rows including row tombstone and static row. (access key is + * rowId) + */ + OFFSETS_VALUES("OffsetsValues"), + /** + * Stores {@link NumericValuesMeta} for {@link NDIType#TOKEN_VALUES} and {@link NDIType#OFFSETS_VALUES}. + */ + GROUP_META("GroupMeta"), + /** + * If present indicates that the per-sstable index build completed successfully + */ + GROUP_COMPLETION_MARKER("GroupComplete", true, true); + + public final String name; + private final boolean perSSTable; + private final boolean marker; + private final boolean encryptable; + + NDIType(String name) + { + this(name, true, false); + } + + NDIType(String name, boolean perSSTable) + { + this(name, perSSTable, false); + } + + NDIType(String name, boolean perSSTable, boolean marker) + { + this.name = name; + this.perSSTable = perSSTable; + this.marker = marker; + this.encryptable = false; + } + + NDIType(String name, boolean perSSTable, boolean marker, boolean encryptable) + { + this.name = name; + this.perSSTable = perSSTable; + this.marker = marker; + this.encryptable = encryptable; + } + + public boolean encryptable() + { + return encryptable; + } + + public boolean perSSTable() + { + return perSSTable; + } + + public boolean completionMarker() + { + return marker; + } + + private boolean perSegment() + { + return !perSSTable && this != META; + } + + public IndexComponent newComponent() + { + assert perSSTable; + String componentName = String.format(PER_SSTABLE_FILE_NAME_FORMAT, name); + + return new IndexComponent(this, componentName); + } + + public IndexComponent newComponent(String indexName) + { + assert !perSSTable; + String componentName = String.format(PER_COLUMN_FILE_NAME_FORMAT, indexName, name); + + return new IndexComponent(this, componentName); + } + + @Override + public String toString() + { + return name; + } + } + + public static final NDIType[] STRING_COMPONENTS = new NDIType[]{ NDIType.TERMS_DATA, NDIType.POSTING_LISTS }; + + private static final NDIType[] NUMERIC_COMPONENTS = new NDIType[]{ NDIType.KD_TREE, NDIType.KD_TREE_POSTING_LISTS }; + + private static final NDIType[] PER_COLUMN_COMPONENTS = new NDIType[]{ NDIType.COLUMN_COMPLETION_MARKER, NDIType.META }; + + private static final NDIType[] NUMERIC_PER_COLUMN_COMPONENTS = ObjectArrays.concat(PER_COLUMN_COMPONENTS, NUMERIC_COMPONENTS, NDIType.class); + + private static final NDIType[] LITERAL_PER_COLUMN_COMPONENTS = ObjectArrays.concat(PER_COLUMN_COMPONENTS, STRING_COMPONENTS, NDIType.class); + + private static final NDIType[] ALL_PER_COLUMN_COMPONENTS = ObjectArrays.concat(NUMERIC_PER_COLUMN_COMPONENTS, STRING_COMPONENTS, NDIType.class); + + public static final IndexComponent TOKEN_VALUES = NDIType.TOKEN_VALUES.newComponent(); + + public static final IndexComponent OFFSETS_VALUES = NDIType.OFFSETS_VALUES.newComponent(); + + public static final IndexComponent GROUP_META = NDIType.GROUP_META.newComponent(); + + public static final IndexComponent GROUP_COMPLETION_MARKER = NDIType.GROUP_COMPLETION_MARKER.newComponent(); + + + /** + * Files that are shared by all storage-attached indexes for each SSTable + */ + public static final List PER_SSTABLE_COMPONENTS = Arrays.asList(GROUP_COMPLETION_MARKER, TOKEN_VALUES, OFFSETS_VALUES, GROUP_META); + + public final IndexComponent termsData, postingLists, meta, groupCompletionMarker, kdTree, kdTreePostingLists, columnCompletionMarker; + + private static final SequentialWriterOption defaultWriterOption = SequentialWriterOption.newBuilder() + .trickleFsync(DatabaseDescriptor.getTrickleFsync()) + .trickleFsyncByteInterval(DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024) + .bufferType(BufferType.OFF_HEAP) + .finishOnClose(true) + .build(); + + public final Descriptor descriptor; + public final String indexName; + + private final SequentialWriterOption writerOption; + private final CompressionParams compressionParams; + + IndexComponents(Descriptor descriptor, SequentialWriterOption sequentialWriterOption, CompressionParams compressionParams) + { + this(null, descriptor, sequentialWriterOption, compressionParams); + } + + @VisibleForTesting + IndexComponents(String indexName, Descriptor descriptor, SequentialWriterOption sequentialWriterOption, CompressionParams compressionParams) + { + this.indexName = indexName; + this.descriptor = descriptor; + this.writerOption = sequentialWriterOption; + + this.compressionParams = compressionParams; + + termsData = NDIType.TERMS_DATA.newComponent(indexName); + postingLists = NDIType.POSTING_LISTS.newComponent(indexName); + meta = NDIType.META.newComponent(indexName); + groupCompletionMarker = NDIType.GROUP_COMPLETION_MARKER.newComponent(); + kdTree = NDIType.KD_TREE.newComponent(indexName); + kdTreePostingLists = NDIType.KD_TREE_POSTING_LISTS.newComponent(indexName); + columnCompletionMarker = NDIType.COLUMN_COMPLETION_MARKER.newComponent(indexName); + } + + /** + * Used to access per-sstable and per-index components + */ + public static IndexComponents create(String indexName, SSTableReader ssTableReader) + { + return create(indexName, ssTableReader.descriptor, CryptoUtils.getCompressionParams(ssTableReader)); + } + + public static IndexComponents create(String indexName, Descriptor descriptor, CompressionParams params) + { + return new IndexComponents(indexName, descriptor, defaultWriterOption, params); + } + + /** + * Returns the sstable {@link Component}s for the specified column index, excluding the shared ones. + */ + public static Set perColumnComponents(String indexName, boolean isLiteral) + { + return components(indexName, isLiteral ? LITERAL_PER_COLUMN_COMPONENTS : NUMERIC_PER_COLUMN_COMPONENTS); + } + + /** + * Used to access per-sstable shared components + */ + public static IndexComponents perSSTable(Descriptor descriptor, CompressionParams params) + { + return new IndexComponents(descriptor, defaultWriterOption, params); + } + + public static IndexComponents perSSTable(SSTableReader ssTableReader) + { + return perSSTable(ssTableReader.descriptor, CryptoUtils.getCompressionParams(ssTableReader)); + } + + /** + * @return true if an index build successfully completed building the per-SSTable + * components for the given SSTable + */ + public static boolean isGroupIndexComplete(Descriptor descriptor) + { + return descriptor.fileFor(GROUP_COMPLETION_MARKER).exists(); + } + + /** + * @return true if an index build successfully completed for the given column index + */ + public static boolean isColumnIndexComplete(Descriptor descriptor, String indexName) + { + return isGroupIndexComplete(descriptor) && descriptor.fileFor(NDIType.COLUMN_COMPLETION_MARKER.newComponent(indexName)).exists(); + } + + /** + * @return true if an index build successfully completed for the given column index but + * the SSTable did not have any indexable rows relating to the index + */ + public static boolean isColumnIndexEmpty(Descriptor descriptor, String indexName) + { + long numIndexFiles = components(indexName, ALL_PER_COLUMN_COMPONENTS).stream().map(descriptor::fileFor).filter(File::exists).count(); + return isColumnIndexComplete(descriptor, indexName) && (numIndexFiles == 1); + } + + /** + * Delete the per-SSTable index files from the filesystem + */ + public static void deletePerSSTableIndexComponents(Descriptor descriptor) + { + PER_SSTABLE_COMPONENTS.stream() + .map(descriptor::fileFor) + .filter(File::exists) + .forEach(IndexComponents::deleteComponent); + } + + private static Set components(String indexName, NDIType... types) + { + Set components = new HashSet<>(types.length); + for (NDIType type : types) + { + components.add(type.newComponent(indexName)); + } + return components; + } + + private static void deleteComponent(File file) + { + logger.debug("Deleting storage attached index component file {}", file); + try + { + IOUtils.deleteFilesIfExist(file.toPath()); + } + catch (IOException e) + { + logger.warn("Unable to delete storage attached index component file {} due to {}.", file, e.getMessage(), e); + } + } + + /** + * @return total size (in bytes) of column index components + */ + public long sizeOfPerColumnComponents() + { + return sizeOf(components(indexName, ALL_PER_COLUMN_COMPONENTS)); + } + + /** + * @return total size (in bytes) of per-SSTable index components + */ + public long sizeOfPerSSTableComponents() + { + return sizeOf(PER_SSTABLE_COMPONENTS); + } + + public long sizeOf(Collection components) + { + return components.stream().map(descriptor::fileFor).filter(File::exists).mapToLong(File::length).sum(); + } + + /** + * A helper method for constructing consistent log messages for specific column indexes. + * + * Example: For the index "idx" in keyspace "ks" on table "tb", calling this method with the raw message + * "Flushing new index segment..." will produce... + * + * "[ks.idx.tb] Flushing new index segment..." + * + * @param message The raw content of a logging message, without information identifying it with an index. + * + * @return A log message with the proper keyspace, table and index name prepended to it. + */ + public String logMessage(String message) + { + // Index names are unique only within a keyspace. + return String.format("[%s.%s.%s] %s", descriptor.ksname, descriptor.cfname, Strings.isNullOrEmpty(indexName) ? "*" : indexName, message); + } + + /** + * Delete the underlying per-column index files from the filesystem. + */ + public void deleteColumnIndex() + { + Stream.of(ALL_PER_COLUMN_COMPONENTS) + .map(type -> type.newComponent(indexName)) + .map(descriptor::fileFor) + .filter(File::exists) + .forEach(IndexComponents::deleteComponent); + } + + public FileHandle createFileHandle(IndexComponent component) + { + return createFileHandle(component, false); + } + + public FileHandle createFileHandle(IndexComponent component, boolean temporary) + { + final File file = temporary ? descriptor.tmpFileFor(component) : descriptor.fileFor(component); + + if (logger.isTraceEnabled()) + { + logger.trace(logMessage("Opening {} file handle for {} ({})"), temporary ? "temporary" : "", file, FBUtilities.prettyPrintMemory(file.length())); + } + + try (final FileHandle.Builder builder = new FileHandle.Builder(file.getAbsolutePath()).mmapped(true)) + { + return builder.complete(); + } + } + + public boolean validatePerSSTableComponentsChecksum() + { + for (IndexComponent component : PER_SSTABLE_COMPONENTS) + { + try + { + validateComponent(component, true); + } + catch (Throwable e) + { + return false; + } + } + return true; + } + + public boolean validatePerColumnComponentsChecksum(boolean isLiteral) + { + try + { + validatePerColumnComponents(isLiteral, true); + return true; + } + catch (Throwable e) + { + logger.warn(logMessage("Checksum validation failed on SSTable {}."), descriptor, e); + return false; + } + } + + public void validatePerSSTableComponents() throws IOException + { + for (IndexComponent component : PER_SSTABLE_COMPONENTS) + { + validateComponent(component, false); + } + } + + public void validatePerColumnComponents(boolean isLiteral) throws IOException + { + validatePerColumnComponents(isLiteral, false); + } + + public IndexInput openInput(FileHandle handle) + { + return IndexInputReader.create(handle); + } + + @SuppressWarnings("resource") + public IndexInput openBlockingInput(IndexComponent component) + { + final File file = descriptor.fileFor(component); + if (logger.isTraceEnabled()) + logger.trace(logMessage("Opening blocking index input for file {} ({})"), file, FBUtilities.prettyPrintMemory(file.length())); + + try (final FileHandle.Builder builder = new FileHandle.Builder(file.getAbsolutePath())) + { + final FileHandle fileHandle = builder.complete(); + final RandomAccessReader randomReader = fileHandle.createReader(); + + return IndexInputReader.create(randomReader, fileHandle::close); + } + } + + public IndexOutputWriter createOutput(IndexComponent component) throws IOException + { + return createOutput(component, false); + } + + public IndexOutputWriter createOutput(IndexComponent component, boolean append) throws IOException + { + return createOutput(component, append, false); + } + + public IndexOutputWriter createOutput(IndexComponent component, boolean append, boolean temporary) throws IOException + { + final File file = temporary ? descriptor.tmpFileFor(component) : descriptor.fileFor(component); + + if (logger.isTraceEnabled()) + logger.trace(logMessage("Creating {} sstable attached index output for component {} on file {}..."), temporary ? "temporary" : "", component, file); + + IndexOutputWriter writer = createOutput(file, component.ndiType.encryptable()); + + if (append) + { + writer.skipBytes(file.length()); + } + + return writer; + } + + public void deleteTemporaryComponents() + { + Stream.of(ALL_PER_COLUMN_COMPONENTS) + .map(type -> type.newComponent(indexName)) + .map(descriptor::tmpFileFor) + .filter(File::exists) + .forEach(IndexComponents::deleteComponent); + } + + public void deleteTemporaryComponent(IndexComponent component) throws IOException + { + final File file = descriptor.tmpFileFor(component); + + if (file.exists()) + if (!file.delete()) + logger.warn("Failed to delete temporary file " + file); + } + + public CompressionParams getCompressionParams() + { + return compressionParams; + } + + @SuppressWarnings("resource") + public IndexOutputWriter createOutput(File file, boolean encryptable) + { + assert writerOption.finishOnClose() : "IndexOutputWriter relies on close() to sync with disk."; + + return new IndexOutputWriter(new IncrementalChecksumSequentialWriter(file)); + } + + @VisibleForTesting + public IndexOutputWriter createOutput(File file) + { + return createOutput(file, false); + } + + public void createGroupCompletionMarker() throws IOException + { + Files.touch(descriptor.fileFor(groupCompletionMarker)); + } + + public void createColumnCompletionMarker() throws IOException + { + Files.touch(descriptor.fileFor(columnCompletionMarker)); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this).add("descriptor", descriptor) + .add("indexName", indexName) + .toString(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + IndexComponents components = (IndexComponents) o; + + if (descriptor != null ? !descriptor.equals(components.descriptor) : components.descriptor != null) + return false; + return indexName != null ? indexName.equals(components.indexName) : components.indexName == null; + } + + @Override + public int hashCode() + { + int result = descriptor != null ? descriptor.hashCode() : 0; + result = 31 * result + (indexName != null ? indexName.hashCode() : 0); + return result; + } + + private void validatePerColumnComponents(boolean isLiteral, boolean checksum) throws IOException + { + MetadataSource source = MetadataSource.loadColumnMetadata(this); + List segments = SegmentMetadata.load(source, null); + + for (IndexComponent component : perColumnComponents(indexName, isLiteral)) + { + if (!component.ndiType.completionMarker()) + { + if (component.ndiType.perSegment()) + { + for (int i = 0; i < segments.size(); i++) + { + SegmentMetadata metadata = segments.get(i); + boolean isLastSegment = i == segments.size() - 1; + + validateSegment(component, metadata, isLastSegment, checksum, false); + } + } + else + { + validateComponent(component, checksum); + } + } + } + } + + @SuppressWarnings("resource") + private void validateSegment(IndexComponent component, SegmentMetadata metadata, boolean isLastSegment, boolean checksum, boolean isEncrypted) throws IOException + { + long offset = metadata.getIndexOffset(component); + long length = metadata.getIndexLength(component); + + try (IndexInput input = openBlockingInput(component)) + { + // Even if the component is encryptable, we still need to check for corruption if it isn't actually encrypted: + if (!component.ndiType.encryptable() || !isEncrypted) + { + // Make sure there isn't any data appended incorrectly after the official end of the file: + if (isLastSegment && input.length() != offset + length) + { + String message = logMessage(String.format("Corrupted last segment! offset (%d) + length (%d) != file "+ + "length (%s) isEncrypted: %s component.ndiType.encryptable: %s component.ndiType: %s", + offset, length, input.length(), isEncrypted, component.ndiType.encryptable(), component.ndiType.toString())); + throw new CorruptIndexException(message, descriptor.toString()); + } + } + + IndexInput slice = input.slice(String.format("%s with offset=%d and length=%d]", input.toString(), offset, length), offset, length); + + if (checksum) + SAICodecUtils.validateChecksum(slice); + else + SAICodecUtils.validate(slice); + } + catch (IOException e) + { + if (logger.isDebugEnabled()) + { + logger.debug(logMessage("Per-segment {} validation failed for index component {} on SSTable {}"), (checksum ? "checksum " : ""), component, descriptor); + } + throw e; + } + } + + private void validateComponent(IndexComponent component, boolean checksum) throws IOException + { + if (!component.ndiType.completionMarker()) + { + try (IndexInput input = openBlockingInput(component)) + { + if (checksum) + SAICodecUtils.validateChecksum(input); + else + SAICodecUtils.validate(input); + } + catch (IOException e) + { + if (logger.isDebugEnabled()) + { + logger.debug(logMessage("{} failed for index component {} on SSTable {}"), (checksum ? "Checksum validation" : "Validation"), component, descriptor); + } + throw e; + } + } + } + + + interface ChecksumWriter + { + long getChecksum(); + } + +// class EncryptedIncrementalChecksumSequentialWriter extends EncryptedSequentialWriter implements ChecksumWriter +// { +// private final CRC32 checksum = new CRC32(); +// +// EncryptedIncrementalChecksumSequentialWriter(File file, ICompressor encryptor) +// { +// super(file, writerOption, encryptor); +// } +// +// @Override +// public void writeByte(int b) throws IOException +// { +// super.writeByte(b); +// checksum.update(b); +// } +// +// @Override +// public void write(byte[] b) throws IOException +// { +// super.write(b); +// checksum.update(b); +// } +// +// @Override +// public void write(byte[] b, int off, int len) throws IOException +// { +// super.write(b, off, len); +// checksum.update(b, off, len); +// } +// +// public long getChecksum() +// { +// return checksum.getValue(); +// } +// } + + class IncrementalChecksumSequentialWriter extends SequentialWriter implements ChecksumWriter + { + private final CRC32 checksum = new CRC32(); + + IncrementalChecksumSequentialWriter(File file) + { + super(file, writerOption); + } + + @Override + public void writeByte(int b) throws IOException + { + super.writeByte(b); + checksum.update(b); + } + + @Override + public void write(byte[] b) throws IOException + { + super.write(b); + checksum.update(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + super.write(b, off, len); + checksum.update(b, off, len); + } + + public long getChecksum() + { + return checksum.getValue(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/IndexInputReader.java b/src/java/org/apache/cassandra/index/sai/disk/io/IndexInputReader.java new file mode 100644 index 000000000000..1ad929fa3bcb --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/IndexInputReader.java @@ -0,0 +1,194 @@ +/* + * 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.index.sai.disk.io; + + +import java.io.IOException; + +import org.apache.cassandra.io.compress.CorruptBlockException; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; + +public class IndexInputReader extends IndexInput +{ + private final RandomAccessReader input; + private final Runnable doOnClose; + + private IndexInputReader(RandomAccessReader input, Runnable doOnClose) + { + super(input.getPath()); + this.input = input; + this.doOnClose = doOnClose; + } + + public static IndexInputReader create(RandomAccessReader input) + { + return new IndexInputReader(input, () -> {}); + } + + static IndexInputReader create(RandomAccessReader input, Runnable doOnClose) + { + return new IndexInputReader(input, doOnClose); + } + + @SuppressWarnings("resource") + static IndexInputReader create(FileHandle handle) + { + RandomAccessReader reader = handle.createReader(); + return new IndexInputReader(reader, () -> {}); + } + + public RandomAccessReader reader() + { + return input; + } + + @Override + public byte readByte() throws IOException + { + return input.readByte(); + } + + @Override + public void readBytes(byte[] bytes, int off, int len) throws IOException + { + try + { + input.readFully(bytes, off, len); + } + catch (CorruptBlockException ex) + { + throw new CorruptIndexException(input.getPath(), "Corrupted block", ex); + } + } + + /** + * Using {@link RandomAccessReader#readShort()} directly is faster than {@link DataInput#readShort()} which calls + * {@link DataInput#readByte()} one by one + */ + @Override + public short readShort() throws IOException + { + try + { + return input.readShort(); + } + catch (CorruptBlockException ex) + { + throw new CorruptIndexException(input.getPath(), "Corrupted block", ex); + } + } + + /** + * Using {@link RandomAccessReader#readInt()} directly is faster than {@link DataInput#readInt()} which + * calls {@link DataInput#readByte()} one by one + */ + @Override + public int readInt() throws IOException + { + try + { + return input.readInt(); + } + catch (CorruptBlockException ex) + { + throw new CorruptIndexException(input.getPath(), "Corrupted block", ex); + } + } + + /** + * Using {@link RandomAccessReader#readLong()} directly is faster than {@link DataInput#readLong()} which + * calls {@link DataInput#readByte()} one by one + */ + @Override + public long readLong() throws IOException + { + try + { + return input.readLong(); + } + catch (CorruptBlockException ex) + { + throw new CorruptIndexException(input.getPath(), "Corrupted block", ex); + } + } + + @Override + public void close() + { + try + { + input.close(); + } + finally + { + doOnClose.run(); + } + } + + @Override + public long getFilePointer() + { + return input.getFilePointer(); + } + + @Override + public void seek(long position) + { + input.seek(position); + } + + @Override + public long length() + { + return input.length(); + } + + @Override + public IndexInput slice(String sliceDescription, long offset, long length) throws CorruptIndexException + { + if (offset < 0 || length < 0 || offset + length > input.length()) + { + throw new CorruptIndexException("Invalid slice! Offset: " + offset + ", Length: " + length + ", Input Length: " + input.length(), this); + } + + return new IndexInputReader(input, doOnClose) + { + @Override + public void seek(long position) + { + input.seek(position + offset); + } + + @Override + public long getFilePointer() + { + return input.getFilePointer() - offset; + } + + @Override + public long length() + { + return length; + } + }; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/IndexOutputWriter.java b/src/java/org/apache/cassandra/index/sai/disk/io/IndexOutputWriter.java new file mode 100644 index 000000000000..f40bb8e68205 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/IndexOutputWriter.java @@ -0,0 +1,115 @@ +/* + * 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.index.sai.disk.io; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; + +import com.google.common.base.MoreObjects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.io.util.SequentialWriter; +import org.apache.lucene.store.IndexOutput; + +public class IndexOutputWriter extends IndexOutput +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final SequentialWriter out; + private boolean closed; + + IndexOutputWriter(SequentialWriter out) + { + super(out.getPath(), out.getPath()); + this.out = out; + } + + public void skipBytes(long length) throws IOException + { + this.out.skipBytes(length); + } + + public String getPath() + { + return out.getPath(); + } + + @Override + public long getChecksum() + { + return ((IndexComponents.ChecksumWriter)out).getChecksum(); + } + + @Override + public long getFilePointer() + { + return out.position(); + } + + @Override + public void writeBytes(byte[] bytes, int offset, int len) throws IOException + { + out.write(bytes, offset, len); + } + + @Override + public void writeByte(byte b) throws IOException + { + out.writeByte(b); + } + + @Override + public void close() throws IOException + { + // IndexOutput#close contract allows any output to be closed multiple times, + // and Lucene does it in few places. SequentialWriter can be closed once. + if (!closed) + { + if (logger.isTraceEnabled()) + { + logger.trace("Closing index output: {}", this); + } + + // The writer should sync its contents to disk before closing... + out.close(); + closed = true; + } + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("path", out.getPath()) + .add("bytesWritten", getFilePointer()) + .add("crc", getChecksum()) + .toString(); + } + + /** + * Returns {@link SequentialWriter} associated with this writer. Convenient when interacting with DSE-DB codebase to + * write files to disk. Note that all bytes written to the returned writer will still contribute to the checksum. + * + * @return {@link SequentialWriter} associated with this writer + */ + public SequentialWriter asSequentialWriter() + { + return out; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/io/RAMIndexOutput.java b/src/java/org/apache/cassandra/index/sai/disk/io/RAMIndexOutput.java new file mode 100644 index 000000000000..b8abd40e4cba --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/io/RAMIndexOutput.java @@ -0,0 +1,77 @@ +/* + * 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.index.sai.disk.io; + +import java.io.IOException; + +import org.apache.lucene.store.GrowableByteArrayDataOutput; +import org.apache.lucene.store.IndexOutput; + +// Use in place of RAMOutputStream (which has monitor locks) +public class RAMIndexOutput extends IndexOutput +{ + protected final GrowableByteArrayDataOutput out; + + public RAMIndexOutput(String name) + { + super("", name); + out = new GrowableByteArrayDataOutput(128); + } + + public byte[] getBytes() + { + return out.getBytes(); + } + + @Override + public long getChecksum() throws IOException + { + throw new UnsupportedOperationException(); + } + + @Override + public long getFilePointer() + { + return out.getPosition(); + } + + @Override + public void writeByte(byte b) + { + out.writeByte(b); + } + + @Override + public void writeBytes(byte[] bytes, int offset, int len) + { + out.writeBytes(bytes, offset, len); + } + + public void writeTo(IndexOutput externalOut) throws IOException + { + externalOut.writeBytes(out.getBytes(), 0, out.getPosition()); + } + + public void reset() + { + out.reset(); + } + + @Override + public void close() throws IOException {} +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/AbstractBlockPackedReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/AbstractBlockPackedReader.java new file mode 100644 index 000000000000..65f36d1cfc43 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/AbstractBlockPackedReader.java @@ -0,0 +1,233 @@ +/* + * 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.index.sai.disk.v1; + +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.index.sai.utils.SeekingRandomAccessInput; +import org.apache.lucene.store.IndexInput; + +public abstract class AbstractBlockPackedReader implements LongArray +{ + private final int blockShift, blockMask; + private final int blockSize; + private final long valueCount; + final byte[] blockBitsPerValue; // package protected for test access + private final SeekingRandomAccessInput input; + + private long prevTokenValue = Long.MIN_VALUE; + private long lastIndex; // the last index visited by token -> row ID searches + + AbstractBlockPackedReader(IndexInput indexInput, byte[] blockBitsPerValue, int blockShift, int blockMask, long sstableRowId, long valueCount) + { + this.blockShift = blockShift; + this.blockMask = blockMask; + this.blockSize = blockMask + 1; + this.valueCount = valueCount; + this.input = new SeekingRandomAccessInput(indexInput); + this.blockBitsPerValue = blockBitsPerValue; + // start searching tokens from current index segment + this.lastIndex = sstableRowId; + } + + protected abstract long blockOffsetAt(int block); + + @Override + public long get(final long index) + { + if (index < 0 || index >= valueCount) + { + throw new IndexOutOfBoundsException(String.format("Index should be between [0, %d), but was %d.", valueCount, index)); + } + + final int block = (int) (index >>> blockShift); + final int idx = (int) (index & blockMask); + final DirectReaders.Reader subReader = DirectReaders.getReaderForBitsPerValue(blockBitsPerValue[block]); + return delta(block, idx) + subReader.get(input, blockOffsetAt(block), idx); + } + + @Override + public long findTokenRowID(long targetValue) + { + // already out of range + if (lastIndex >= valueCount) + return -1; + + // We keep track previous returned value in lastIndex, so searching backward will not return correct result. + // Also it's logically wrong to search backward during token iteration in PostingListRangeIterator. + if (targetValue < prevTokenValue) + throw new IllegalArgumentException(String.format("%d is smaller than prev token value %d", targetValue, prevTokenValue)); + prevTokenValue = targetValue; + + int blockIndex = binarySearchBlockMinValues(targetValue); + + // We need to check next block's min value on an exact match. + boolean exactMatch = blockIndex >= 0; + + if (blockIndex < 0) + { + // A non-exact match, which is the negative index of the first value greater than the target. + // For example, searching for 4 against min values [3,3,5,7] produces -2, which we convert to 2. + blockIndex = -blockIndex; + } + + if (blockIndex > 0) + { + // Start at the previous block, because there could be duplicate values in the previous block. + // For example, with block 1: [1,2,3,3] & block 2: [3,3,5,7], binary search for 3 would find + // block 2, but we need to start from block 1 and search both. + // In case non-exact match, we need to pivot left as target is less than next block's min. + blockIndex--; + } + + // Find the global (not block-specific) index of the target token, which is equivalent to its row ID: + lastIndex = findBlockRowID(targetValue, blockIndex, exactMatch); + return lastIndex >= valueCount ? -1 : lastIndex; + } + + /** + * + * @return a positive block index for an exact match, or a negative one for a non-exact match + */ + private int binarySearchBlockMinValues(long targetValue) + { + int high = Math.toIntExact(blockBitsPerValue.length) - 1; + + // Assume here that we'll never move backward through the blocks: + int low = Math.toIntExact(lastIndex >> blockShift); + + // Short-circuit the search if the target is in current block: + if (low + 1 <= high) + { + long cmp = Long.compare(targetValue, delta(low + 1, 0)); + + if (cmp == 0) + { + // We have an exact match, so return the index of the next block, which means we'll start + // searching from the current one and also inspect the first value of the next block. + return low + 1; + } + else if (cmp < 0) + { + // We're in the same block. Indicate a non-exact match, and this value will be both + // negated and then decremented to wind up at the current value of "low" here. + return -low - 1; + } + + // The target is greater than the next block's min value, so advance to that + // block before starting the usual search... + low++; + } + + while (low <= high) + { + int mid = low + ((high - low) >> 1); + + long midVal = delta(mid, 0); + + if (midVal < targetValue) + { + low = mid + 1; + } + else if (midVal > targetValue) + { + high = mid - 1; + } + else + { + // target found, but we need to check for duplicates + if (mid > 0 && delta(mid - 1, 0) == targetValue) + { + // there are duplicates, pivot left + high = mid - 1; + } + else + { + // no duplicates + return mid; + } + } + } + + return -low; // no exact match found + } + + private long findBlockRowID(long targetValue, long blockIdx, boolean exactMatch) + { + // Calculate the global offset for the selected block: + long offset = blockIdx << blockShift; + + // Resume from previous index if it's larger than offset + long low = Math.max(lastIndex, offset); + + // The high is either the last local index in the block, or something smaller if the block isn't full: + long high = Math.min(offset + blockSize - 1 + (exactMatch ? 1 : 0), valueCount - 1); + + return binarySearchBlock(targetValue, low, high); + } + + /** + * binary search target value between low and high. + * + * @return index if exact match is found, or *positive* insertion point if no exact match is found. + */ + private long binarySearchBlock(long target, long low, long high) + { + while (low <= high) + { + long mid = low + ((high - low) >> 1); + + long midVal = get(mid); + + if (midVal < target) + { + low = mid + 1; + // future rowId cannot be smaller than mid as long as next token not smaller than current token. + lastIndex = mid; + } + else if (midVal > target) + { + high = mid - 1; + } + else + { + // target found, but we need to check for duplicates + if (mid > 0 && get(mid - 1) == target) + { + // there are duplicates, pivot left + high = mid - 1; + } + else + { + // exact match and no duplicates + return mid; + } + } + } + + // target not found + return low; + } + + @Override + public long length() + { + return valueCount; + } + + abstract long delta(int block, int idx); +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/AbstractBlockPackedWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/AbstractBlockPackedWriter.java new file mode 100644 index 000000000000..ca8a0df6c3ea --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/AbstractBlockPackedWriter.java @@ -0,0 +1,117 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.cassandra.index.sai.disk.io.RAMIndexOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.packed.DirectWriter; + +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.checkBlockSize; + +/** + * Modified copy of {@link org.apache.lucene.util.packed.AbstractBlockPackedWriter} to use {@link DirectWriter} for + * optimised reads that doesn't require seeking through the whole file to open a thread-exclusive reader. + */ +abstract class AbstractBlockPackedWriter +{ + static final int MIN_BLOCK_SIZE = 64; + static final int MAX_BLOCK_SIZE = 1 << (30 - 3); + static final int MIN_VALUE_EQUALS_0 = 1; + static final int BPV_SHIFT = 1; + + protected final IndexOutput out; + protected final long[] values; + protected int off; + protected boolean finished; + + final RAMIndexOutput blockMetaWriter; + + AbstractBlockPackedWriter(IndexOutput out, int blockSize) + { + checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE); + this.out = out; + this.blockMetaWriter = new RAMIndexOutput("NumericValuesMeta"); + values = new long[blockSize]; + } + + private void checkNotFinished() + { + if (finished) + { + throw new IllegalStateException(String.format("[%s] Writer already finished!", out.getName())); + } + } + + /** + * Append a new long. + */ + public void add(long l) throws IOException + { + checkNotFinished(); + if (off == values.length) + { + flush(); + } + values[off++] = l; + } + + + /** + * Flush all buffered data to disk. This instance is not usable anymore + * after this method has been called. + * + * @return a file offset to the block metadata + */ + public long finish() throws IOException + { + checkNotFinished(); + if (off > 0) + { + flush(); + } + final long fp = out.getFilePointer(); + blockMetaWriter.writeTo(out); + finished = true; + return fp; + } + + protected abstract void flush() throws IOException; + + void writeValues(int numValues, int bitsPerValue) throws IOException + { + final DirectWriter writer = DirectWriter.getInstance(out, numValues, bitsPerValue); + for (int i = 0; i < numValues; ++i) + { + writer.add(values[i]); + } + writer.finish(); + } + + void writeVLong(IndexOutput out, long i) throws IOException + { + int k = 0; + while ((i & ~0x7FL) != 0L && k++ < 8) + { + out.writeByte((byte) ((i & 0x7FL) | 0x80L)); + i >>>= 7; + } + out.writeByte((byte) i); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/BKDPostingsIndex.java b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDPostingsIndex.java new file mode 100644 index 000000000000..6d5e6d6f8406 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDPostingsIndex.java @@ -0,0 +1,95 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import com.carrotsearch.hppc.IntLongHashMap; +import com.carrotsearch.hppc.IntLongMap; +import org.apache.cassandra.index.sai.disk.io.IndexInputReader; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.github.jamm.MemoryLayoutSpecification; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.validate; + +/** + * Mapping between node ID and an offset to its auxiliary posting list (containing every row id from all leaves + * reachable from that node. See {@link OneDimBKDPostingsWriter}). + */ +class BKDPostingsIndex +{ + private final int size; + public final IntLongMap index = new IntLongHashMap(); + + @SuppressWarnings("resource") + BKDPostingsIndex(FileHandle postingsFileHandle, long filePosition) throws IOException + { + try (final RandomAccessReader reader = postingsFileHandle.createReader()) + { + final IndexInputReader input = IndexInputReader.create(reader); + validate(input); + input.seek(filePosition); + + size = input.readVInt(); + + for (int x = 0; x < size; x++) + { + final int node = input.readVInt(); + final long filePointer = input.readVLong(); + + index.put(node, filePointer); + } + } + } + + public long memoryUsage() + { + // IntLongHashMap uses two arrays: one for keys, one for values. + return MemoryLayoutSpecification.sizeOfArray(index.size(), 4L) + + MemoryLayoutSpecification.sizeOfArray(index.size(), 8L); + } + + /** + * Returns true if given node ID has an auxiliary posting list. + */ + boolean exists(int nodeID) + { + checkArgument(nodeID > 0); + return index.containsKey(nodeID); + } + + /** + * Returns an offset within the bkd postings file to the begining of the blocks summary of given node's auxiliary + * posting list. + * + * @throws IllegalArgumentException when given nodeID doesn't have an auxiliary posting list. Check first with + * {@link #exists(int)} + */ + long getPostingsFilePointer(int nodeID) + { + checkArgument(exists(nodeID)); + return index.get(nodeID); + } + + int size() + { + return size; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/BKDReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDReader.java new file mode 100644 index 000000000000..9a5e7452fc15 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDReader.java @@ -0,0 +1,809 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.agrona.collections.LongArrayList; +import org.apache.cassandra.index.sai.QueryContext; +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.cassandra.index.sai.disk.io.CryptoUtils; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.metrics.QueryEventListener; +import org.apache.cassandra.index.sai.utils.AbortedOperationException; +import org.apache.cassandra.index.sai.utils.AbstractIterator; +import org.apache.cassandra.index.sai.utils.SeekingRandomAccessInput; +import org.apache.cassandra.io.compress.ICompressor; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Throwables; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.PointValues.Relation; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.FixedBitSet; +import org.apache.lucene.util.FutureArrays; +import org.apache.lucene.util.packed.DirectWriter; + +/** + * Handles intersection of a multi-dimensional shape in byte[] space with a block KD-tree previously written with + * {@link BKDWriter}. + */ +public class BKDReader extends TraversingBKDReader implements Closeable +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private static final Comparator COMPARATOR = Comparator.comparingLong(PostingList.PeekablePostingList::peek); + + private final FileHandle postingsFile, kdtreeFile; + private final BKDPostingsIndex postingsIndex; + private final ICompressor compressor; + private final DirectReaders.Reader leafOrderMapReader; + + /** + * Performs a blocking read. + */ + public BKDReader(IndexComponents indexComponents, + FileHandle kdtreeFile, + long bkdIndexRoot, + FileHandle postingsFile, + long bkdPostingsRoot) throws IOException + { + super(indexComponents, kdtreeFile, bkdIndexRoot); + this.postingsFile = postingsFile; + this.kdtreeFile = kdtreeFile; + this.postingsIndex = new BKDPostingsIndex(postingsFile, bkdPostingsRoot); + this.compressor = null; + final byte bits = (byte) DirectWriter.unsignedBitsRequired(maxPointsInLeafNode - 1); + leafOrderMapReader = DirectReaders.getReaderForBitsPerValue(bits); + } + + public interface DocMapper + { + long oldToNew(long rowID); + } + + private TreeMap getLeafOffsets() + { + final TreeMap map = new TreeMap(); + final PackedIndexTree index = new PackedIndexTree(); + getLeafOffsets(index, map); + return map; + } + + private void getLeafOffsets(final IndexTree index, TreeMap map) + { + if (index.isLeafNode()) + { + if (index.nodeExists()) + { + map.put(index.getLeafBlockFP(), index.getNodeID()); + } + } + else + { + index.pushLeft(); + getLeafOffsets(index, map); + index.pop(); + + index.pushRight(); + getLeafOffsets(index, map); + index.pop(); + } + } + + @VisibleForTesting + public IteratorState iteratorState() throws IOException + { + return new IteratorState((rowID) -> rowID); + } + + public IteratorState iteratorState(DocMapper docMapper) throws IOException + { + return new IteratorState(docMapper); + } + + public class IteratorState extends AbstractIterator implements Comparable, Closeable + { + final IndexInput bkdInput; + final IndexInput bkdPostingsInput; + final byte[] packedValues = new byte[maxPointsInLeafNode * packedBytesLength]; + private int leaf, leafPointCount, leafPointIndex = -1; + final LongArrayList tempPostings = new LongArrayList(); + final long[] postings = new long[maxPointsInLeafNode]; + final DocMapper docMapper; + public final byte[] scratch; + final Iterator> iterator; + + public IteratorState(DocMapper docMapper) throws IOException + { + this.docMapper = docMapper; + + scratch = new byte[packedBytesLength]; + + final long firstLeafFilePointer = getMinLeafBlockFP(); + bkdInput = indexComponents.openInput(kdtreeFile); + bkdPostingsInput = indexComponents.openInput(postingsFile); + bkdInput.seek(firstLeafFilePointer); + + final TreeMap leafNodeToLeafFP = getLeafOffsets(); + + // init the first leaf + iterator = leafNodeToLeafFP.entrySet().iterator(); + final Map.Entry entry = iterator.next(); + leafPointCount = readLeaf(entry.getKey(), entry.getValue(), bkdInput, packedValues, bkdPostingsInput, postings, tempPostings); + } + + @Override + public void close() + { + FileUtils.closeQuietly(bkdInput, bkdPostingsInput); + } + + @Override + public int compareTo(final IteratorState other) + { + final int cmp = FutureArrays.compareUnsigned(scratch, 0, packedBytesLength, other.scratch, 0, packedBytesLength); + if (cmp == 0) + { + final long rowid1 = next; + final long rowid2 = other.next; + return Long.compare(rowid1, rowid2); + } + return cmp; + } + + @Override + protected Long computeNext() + { + while (true) + { + if (leafPointIndex == leafPointCount - 1) + { + leaf++; + if (leaf == numLeaves && leafPointIndex == leafPointCount - 1) + { + return endOfData(); + } + final Map.Entry entry = iterator.next(); + try + { + leafPointCount = readLeaf(entry.getKey(), entry.getValue(), bkdInput, packedValues, bkdPostingsInput, postings, tempPostings); + } + catch (IOException e) + { + logger.error("Failed to read leaf during BKDTree merger", e); + throw new RuntimeException("Failed to read leaf during BKDTree merger", e); + } + leafPointIndex = -1; + } + + leafPointIndex++; + + System.arraycopy(packedValues, leafPointIndex * packedBytesLength, scratch, 0, packedBytesLength); + return docMapper.oldToNew(postings[leafPointIndex]); + } + } + } + + @SuppressWarnings("resource") + public int readLeaf(long filePointer, + int nodeID, + final IndexInput bkdInput, + final byte[] packedValues, + final IndexInput bkdPostingsInput, + long[] postings, + LongArrayList tempPostings) throws IOException + { + bkdInput.seek(filePointer); + final int count = bkdInput.readVInt(); + // loading doc ids occurred here prior + final int orderMapLength = bkdInput.readVInt(); + final long orderMapPointer = bkdInput.getFilePointer(); + + // order of the values in the posting list + final short[] origIndex = new short[maxPointsInLeafNode]; + + final int[] commonPrefixLengths = new int[numDims]; + final byte[] scratchPackedValue1 = new byte[packedBytesLength]; + + final SeekingRandomAccessInput randoInput = new SeekingRandomAccessInput(bkdInput); + for (int x = 0; x < count; x++) + { + final short idx = (short) LeafOrderMap.getValue(randoInput, orderMapPointer, x, leafOrderMapReader); + origIndex[x] = idx; + } + + IndexInput leafInput = bkdInput; + + // reused byte arrays for the decompression of leaf values + final BytesRef uncompBytes = new BytesRef(new byte[16]); + final BytesRef compBytes = new BytesRef(new byte[16]); + + // seek beyond the ordermap + leafInput.seek(orderMapPointer + orderMapLength); + + if (compressor != null) + { + // This should not throw WouldBlockException, even though we're on a TPC thread, because the + // secret key used by the underlying encryptor should be loaded at reader construction time. + leafInput = CryptoUtils.uncompress(bkdInput, compressor, compBytes, uncompBytes); + } + + final IntersectVisitor visitor = new IntersectVisitor() { + int i = 0; + + @Override + public boolean visit(byte[] packedValue) + { + System.arraycopy(packedValue, 0, packedValues, i * packedBytesLength, packedBytesLength); + i++; + return true; + } + + @Override + public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { + return Relation.CELL_CROSSES_QUERY; + } + }; + + visitDocValues(commonPrefixLengths, scratchPackedValue1, leafInput, count, visitor, null, origIndex); + + if (postingsIndex.exists(nodeID)) + { + final long pointer = postingsIndex.getPostingsFilePointer(nodeID); + final PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(bkdPostingsInput, pointer); + final PostingsReader postingsReader = new PostingsReader(bkdPostingsInput, summary, QueryEventListener.PostingListEventListener.NO_OP); + + tempPostings.clear(); + + // gather the postings into tempPostings + while (true) + { + final long rowid = postingsReader.nextPosting(); + if (rowid == PostingList.END_OF_STREAM) break; + tempPostings.add(rowid); + } + + // put the postings into the array according the origIndex + for (int x = 0; x < tempPostings.size(); x++) + { + int idx = origIndex[x]; + final long rowid = tempPostings.get(idx); + + postings[x] = rowid; + } + } + else + { + throw new IllegalStateException(); + } + return count; + } + + public static int openPerIndexFiles() + { + // kd-tree, posting lists file + return 2; + } + + @Override + public void close() + { + try + { + super.close(); + } + finally + { + postingsFile.close(); + } + } + + @SuppressWarnings("resource") + public PostingList intersect(IntersectVisitor visitor, QueryEventListener.BKDIndexEventListener listener, QueryContext context) + { + Relation relation = visitor.compare(minPackedValue, maxPackedValue); + + if (relation == Relation.CELL_OUTSIDE_QUERY) + { + listener.onIntersectionEarlyExit(); + return null; + } + + listener.onSegmentHit(); + IndexInput bkdInput = indexComponents.openInput(indexFile); + IndexInput postingsInput = indexComponents.openInput(postingsFile); + IndexInput postingsSummaryInput = indexComponents.openInput(postingsFile); + PackedIndexTree index = new PackedIndexTree(); + + Intersection completable = + relation == Relation.CELL_INSIDE_QUERY ? + new Intersection(bkdInput, postingsInput, postingsSummaryInput, index, listener, context) : + new FilteringIntersection(bkdInput, postingsInput, postingsSummaryInput, index, visitor, listener, context); + + return completable.execute(); + } + + /** + * Synchronous intersection of an multi-dimensional shape in byte[] space with a block KD-tree + * previously written with {@link BKDWriter}. + */ + class Intersection + { + private final Stopwatch queryExecutionTimer = Stopwatch.createStarted(); + final QueryContext context; + + final IndexInput bkdInput; + final IndexInput postingsInput; + final IndexInput postingsSummaryInput; + final IndexTree index; + final QueryEventListener.BKDIndexEventListener listener; + + Intersection(IndexInput bkdInput, IndexInput postingsInput, IndexInput postingsSummaryInput, + IndexTree index, QueryEventListener.BKDIndexEventListener listener, QueryContext context) + { + this.bkdInput = bkdInput; + this.postingsInput = postingsInput; + this.postingsSummaryInput = postingsSummaryInput; + this.index = index; + this.listener = listener; + this.context = context; + } + + public PostingList execute() + { + try + { + PriorityQueue postingLists = new PriorityQueue<>(100, COMPARATOR); + executeInternal(postingLists); + + FileUtils.closeQuietly(bkdInput); + + return mergePostings(postingLists); + } + catch (Throwable t) + { + if (!(t instanceof AbortedOperationException)) + logger.error(indexComponents.logMessage("kd-tree intersection failed on {}"), indexFile.path(), t); + + closeOnException(); + throw Throwables.cleaned(t); + } + } + + protected void executeInternal(final PriorityQueue postingLists) throws IOException + { + collectPostingLists(postingLists); + } + + protected void closeOnException() + { + FileUtils.closeQuietly(bkdInput, postingsInput, postingsSummaryInput); + } + + protected PostingList mergePostings(PriorityQueue postingLists) + { + final long elapsedMicros = queryExecutionTimer.stop().elapsed(TimeUnit.MICROSECONDS); + + listener.onIntersectionComplete(elapsedMicros, TimeUnit.MICROSECONDS); + listener.postingListsHit(postingLists.size()); + + if (postingLists.isEmpty()) + { + FileUtils.closeQuietly(postingsInput, postingsSummaryInput); + return null; + } + else + { + if (logger.isTraceEnabled()) + logger.trace(indexComponents.logMessage("[{}] Intersection completed in {} microseconds. {} leaf and internal posting lists hit."), + indexFile.path(), elapsedMicros, postingLists.size()); + return MergePostingList.merge(postingLists, () -> FileUtils.close(postingsInput, postingsSummaryInput)); + } + } + + public void collectPostingLists(PriorityQueue postingLists) throws IOException + { + context.checkpoint(); + + final int nodeID = index.getNodeID(); + + // if there is pre-built posting for entire subtree + if (postingsIndex.exists(nodeID)) + { + postingLists.add(initPostingReader(postingsIndex.getPostingsFilePointer(nodeID)).peekable()); + return; + } + + Preconditions.checkState(!index.isLeafNode(), "Leaf node %s does not have kd-tree postings.", index.getNodeID()); + + // Recurse on left sub-tree: + index.pushLeft(); + collectPostingLists(postingLists); + index.pop(); + + // Recurse on right sub-tree: + index.pushRight(); + collectPostingLists(postingLists); + index.pop(); + } + + private PostingList initPostingReader(long offset) throws IOException + { + final PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(postingsSummaryInput, offset); + return new PostingsReader(postingsInput, summary, listener.postingListEventListener()); + } + } + + /** + * Modified copy of BKDReader#visitDocValues() + */ + private int visitDocValues(int[] commonPrefixLengths, + byte[] scratchPackedValue1, + IndexInput in, + int count, + IntersectVisitor visitor, + FixedBitSet[] holder, + final short[] origIndex) throws IOException + { + readCommonPrefixes(commonPrefixLengths, scratchPackedValue1, in); + + int compressedDim = readCompressedDim(in); + if (compressedDim == -1) + { + return visitRawDocValues(commonPrefixLengths, scratchPackedValue1, in, count, visitor, holder, origIndex); + } + else + { + return visitCompressedDocValues(commonPrefixLengths, scratchPackedValue1, in, count, visitor, compressedDim, holder, origIndex); + } + } + + /** + * Modified copy of {@link org.apache.lucene.util.bkd.BKDReader#readCompressedDim(IndexInput)} + */ + @SuppressWarnings("JavadocReference") + private int readCompressedDim(IndexInput in) throws IOException + { + int compressedDim = in.readByte(); + if (compressedDim < -1 || compressedDim >= numDims) + { + throw new CorruptIndexException(String.format("Dimension should be in the range [-1, %d), but was %d.", numDims, compressedDim), in); + } + return compressedDim; + } + + /** + * Modified copy of BKDReader#visitCompressedDocValues() + */ + private int visitCompressedDocValues(int[] commonPrefixLengths, + byte[] scratchPackedValue, + IndexInput in, + int count, + IntersectVisitor visitor, + int compressedDim, + FixedBitSet[] holder, + final short[] origIndex) throws IOException + { + // the byte at `compressedByteOffset` is compressed using run-length compression, + // other suffix bytes are stored verbatim + final int compressedByteOffset = compressedDim * bytesPerDim + commonPrefixLengths[compressedDim]; + commonPrefixLengths[compressedDim]++; + int i, collected = 0; + + final FixedBitSet bitSet; + if (holder != null) + { + bitSet = new FixedBitSet(maxPointsInLeafNode); + } + else + { + bitSet = null; + } + + for (i = 0; i < count; ) + { + scratchPackedValue[compressedByteOffset] = in.readByte(); + final int runLen = Byte.toUnsignedInt(in.readByte()); + for (int j = 0; j < runLen; ++j) + { + for (int dim = 0; dim < numDims; dim++) + { + int prefix = commonPrefixLengths[dim]; + in.readBytes(scratchPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix); + } + final int rowIDIndex = origIndex[i + j]; + if (visitor.visit(scratchPackedValue)) + { + if (bitSet != null) bitSet.set(rowIDIndex); + collected++; + } + } + i += runLen; + } + if (i != count) + { + throw new CorruptIndexException(String.format("Expected %d sub-blocks but read %d.", count, i), in); + } + + if (holder != null) + { + holder[0] = bitSet; + } + + return collected; + } + + /** + * Modified copy of BKDReader#visitRawDocValues() + */ + private int visitRawDocValues(int[] commonPrefixLengths, + byte[] scratchPackedValue, + IndexInput in, + int count, + IntersectVisitor visitor, + FixedBitSet[] holder, + final short[] origIndex) throws IOException + { + final FixedBitSet bitSet; + if (holder != null) + { + bitSet = new FixedBitSet(maxPointsInLeafNode); + } + else + { + bitSet = null; + } + + int collected = 0; + for (int i = 0; i < count; ++i) + { + for (int dim = 0; dim < numDims; dim++) + { + int prefix = commonPrefixLengths[dim]; + in.readBytes(scratchPackedValue, dim * bytesPerDim + prefix, bytesPerDim - prefix); + } + final int rowIDIndex = origIndex[i]; + if (visitor.visit(scratchPackedValue)) + { + if (bitSet != null) bitSet.set(rowIDIndex); + + collected++; + } + } + if (holder != null) + { + holder[0] = bitSet; + } + return collected; + } + + /** + * Copy of BKDReader#readCommonPrefixes() + */ + private void readCommonPrefixes(int[] commonPrefixLengths, byte[] scratchPackedValue, IndexInput in) throws IOException + { + for (int dim = 0; dim < numDims; dim++) + { + int prefix = in.readVInt(); + commonPrefixLengths[dim] = prefix; + if (prefix > 0) + { +// System.out.println("dim * bytesPerDim="+(dim * bytesPerDim)+" prefix="+prefix+" numDims="+numDims); + in.readBytes(scratchPackedValue, dim * bytesPerDim, prefix); + } + } + } + + private class FilteringIntersection extends Intersection + { + private final IntersectVisitor visitor; + private final byte[] scratchPackedValue1; + private final int[] commonPrefixLengths; + private final short[] origIndex; + + // reused byte arrays for the decompression of leaf values + private final BytesRef uncompBytes = new BytesRef(new byte[16]); + private final BytesRef compBytes = new BytesRef(new byte[16]); + + FilteringIntersection(IndexInput bkdInput, IndexInput postingsInput, IndexInput postingsSummaryInput, + IndexTree index, IntersectVisitor visitor, + QueryEventListener.BKDIndexEventListener listener, QueryContext context) + { + super(bkdInput, postingsInput, postingsSummaryInput, index, listener, context); + this.visitor = visitor; + this.commonPrefixLengths = new int[numDims]; + this.scratchPackedValue1 = new byte[packedBytesLength]; + this.origIndex = new short[maxPointsInLeafNode]; + } + + @Override + public void executeInternal(final PriorityQueue postingLists) throws IOException + { + collectPostingLists(postingLists, minPackedValue, maxPackedValue); + } + + public void collectPostingLists(PriorityQueue postingLists, byte[] cellMinPacked, byte[] cellMaxPacked) throws IOException + { + context.checkpoint(); + + final Relation r = visitor.compare(cellMinPacked, cellMaxPacked); + + if (r == Relation.CELL_OUTSIDE_QUERY) + { + // This cell is fully outside of the query shape: stop recursing + return; + } + + if (r == Relation.CELL_INSIDE_QUERY) + { + // This cell is fully inside of the query shape: recursively add all points in this cell without filtering + super.collectPostingLists(postingLists); + return; + } + + if (index.isLeafNode()) + { + if (index.nodeExists()) + filterLeaf(postingLists); + return; + } + + visitNode(postingLists, cellMinPacked, cellMaxPacked); + } + + @SuppressWarnings("resource") + void filterLeaf(PriorityQueue postingLists) throws IOException + { + bkdInput.seek(index.getLeafBlockFP()); + + final int count = bkdInput.readVInt(); + + // loading doc ids occurred here prior + + final FixedBitSet[] holder = new FixedBitSet[1]; + + final int orderMapLength = bkdInput.readVInt(); + + final long orderMapPointer = bkdInput.getFilePointer(); + + final SeekingRandomAccessInput randoInput = new SeekingRandomAccessInput(bkdInput); + for (int x = 0; x < count; x++) + { + origIndex[x] = (short) LeafOrderMap.getValue(randoInput, orderMapPointer, x, leafOrderMapReader); + } + + // seek beyond the ordermap + bkdInput.seek(orderMapPointer + orderMapLength); + + IndexInput leafInput = bkdInput; + + if (compressor != null) + { + // This should not throw WouldBlockException, even though we're on a TPC thread, because the + // secret key used by the underlying encryptor should be loaded at reader construction time. + leafInput = CryptoUtils.uncompress(bkdInput, compressor, compBytes, uncompBytes); + } + + visitDocValues(commonPrefixLengths, scratchPackedValue1, leafInput, count, visitor, holder, origIndex); + + final int nodeID = index.getNodeID(); + + if (postingsIndex.exists(nodeID) && holder[0].cardinality() > 0) + { + final long pointer = postingsIndex.getPostingsFilePointer(nodeID); + postingLists.add(initFilteringPostingReader(pointer, holder[0]).peekable()); + } + } + + void visitNode(PriorityQueue postingLists, byte[] cellMinPacked, byte[] cellMaxPacked) throws IOException + { + int splitDim = index.getSplitDim(); + assert splitDim >= 0 : "splitDim=" + splitDim; + assert splitDim < numDims; + + byte[] splitPackedValue = index.getSplitPackedValue(); + BytesRef splitDimValue = index.getSplitDimValue(); + assert splitDimValue.length == bytesPerDim; + + // make sure cellMin <= splitValue <= cellMax: + assert FutureArrays.compareUnsigned(cellMinPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) <= 0 : "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims; + assert FutureArrays.compareUnsigned(cellMaxPacked, splitDim * bytesPerDim, splitDim * bytesPerDim + bytesPerDim, splitDimValue.bytes, splitDimValue.offset, splitDimValue.offset + bytesPerDim) >= 0 : "bytesPerDim=" + bytesPerDim + " splitDim=" + splitDim + " numDims=" + numDims; + + // Recurse on left sub-tree: + System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength); + System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim * bytesPerDim, bytesPerDim); + + index.pushLeft(); + collectPostingLists(postingLists, cellMinPacked, splitPackedValue); + index.pop(); + + // Restore the split dim value since it may have been overwritten while recursing: + System.arraycopy(splitPackedValue, splitDim * bytesPerDim, splitDimValue.bytes, splitDimValue.offset, bytesPerDim); + // Recurse on right sub-tree: + System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength); + System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim * bytesPerDim, bytesPerDim); + index.pushRight(); + collectPostingLists(postingLists, splitPackedValue, cellMaxPacked); + index.pop(); + } + + private PostingList initFilteringPostingReader(long offset, FixedBitSet filter) throws IOException + { + final PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(postingsSummaryInput, offset); + return initFilteringPostingReader(filter, summary); + } + + @SuppressWarnings("resource") + private PostingList initFilteringPostingReader(FixedBitSet filter, PostingsReader.BlocksSummary header) throws IOException + { + PostingsReader postingsReader = new PostingsReader(postingsInput, header, listener.postingListEventListener()); + return new FilteringPostingList(filter, postingsReader); + } + } + + public int getNumDimensions() + { + return numDims; + } + + public int getBytesPerDimension() + { + return bytesPerDim; + } + + public long getPointCount() + { + return pointCount; + } + + /** + * We recurse the BKD tree, using a provided instance of this to guide the recursion. + */ + public interface IntersectVisitor + { + /** + * Called for all values in a leaf cell that crosses the query. The consumer + * should scrutinize the packedValue to decide whether to accept it. In the 1D case, + * values are visited in increasing order, and in the case of ties, in increasing order + * by segment row ID. + */ + boolean visit(byte[] packedValue); + + /** + * Called for non-leaf cells to test how the cell relates to the query, to + * determine how to further recurse down the tree. + */ + Relation compare(byte[] minPackedValue, byte[] maxPackedValue); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/BKDTreeRamBuffer.java b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDTreeRamBuffer.java new file mode 100644 index 000000000000..6483ad17bc7b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDTreeRamBuffer.java @@ -0,0 +1,194 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import com.google.common.base.Preconditions; + +import org.apache.cassandra.index.sai.disk.MutableOneDimPointValues; +import org.apache.lucene.codecs.MutablePointValues; +import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.ByteBlockPool; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.Counter; +import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedLongValues; + +/** + * On-heap buffer for point values that provides a sortable view of itself as {@link MutablePointValues}. + */ +public class BKDTreeRamBuffer implements Accountable +{ + private final Counter bytesUsed; + private final ByteBlockPool bytes; + private final int pointDimensionCount, pointNumBytes; + private final int packedBytesLength; + private final byte[] packedValue; + private final PackedLongValues.Builder docIDsBuilder; + private int numPoints; + private int numRows; + private int lastSegmentRowID = -1; + private boolean closed = false; + + public BKDTreeRamBuffer(int pointDimensionCount, int pointNumBytes) + { + this.bytesUsed = Counter.newCounter(); + this.pointDimensionCount = pointDimensionCount; + this.pointNumBytes = pointNumBytes; + + this.bytes = new ByteBlockPool(new ByteBlockPool.DirectTrackingAllocator(bytesUsed)); + + packedValue = new byte[pointDimensionCount * pointNumBytes]; + packedBytesLength = pointDimensionCount * pointNumBytes; + + docIDsBuilder = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT); + bytesUsed.addAndGet(docIDsBuilder.ramBytesUsed()); + } + + @Override + public long ramBytesUsed() + { + return bytesUsed.get(); + } + + public int numRows() + { + return numRows; + } + + public long addPackedValue(int segmentRowId, BytesRef value) + { + ensureOpen(); + + if (value.length != packedBytesLength) + { + throw new IllegalArgumentException("The value has length=" + value.length + " but should be " + pointDimensionCount * pointNumBytes); + } + + long startingBytesUsed = bytesUsed.get(); + long startingDocIDsBytesUsed = docIDsBuilder.ramBytesUsed(); + + docIDsBuilder.add(segmentRowId); + bytes.append(value); + + if (segmentRowId != lastSegmentRowID) + { + numRows++; + lastSegmentRowID = segmentRowId; + } + + numPoints++; + + long docIDsAllocatedBytes = docIDsBuilder.ramBytesUsed() - startingDocIDsBytesUsed; + long endingBytesAllocated = bytesUsed.addAndGet(docIDsAllocatedBytes); + + return endingBytesAllocated - startingBytesUsed; + } + + public MutableOneDimPointValues asPointValues() + { + ensureOpen(); + // building packed longs is destructive + closed = true; + final PackedLongValues docIDs = docIDsBuilder.build(); + return new MutableOneDimPointValues() + { + final int[] ords = new int[numPoints]; + + { + for (int i = 0; i < numPoints; ++i) + { + ords[i] = i; + } + } + + @Override + public void getValue(int i, BytesRef packedValue) + { + final long offset = (long) packedBytesLength * (long) ords[i]; + packedValue.length = packedBytesLength; + bytes.setRawBytesRef(packedValue, offset); + } + + @Override + public byte getByteAt(int i, int k) + { + final long offset = (long) packedBytesLength * (long) ords[i] + (long) k; + + return bytes.readByte(offset); + } + + @Override + public int getDocID(int i) + { + return Math.toIntExact(docIDs.get(ords[i])); + } + + @Override + public void swap(int i, int j) + { + int tmp = ords[i]; + ords[i] = ords[j]; + ords[j] = tmp; + } + + @Override + public void intersect(IntersectVisitor visitor) throws IOException + { + final BytesRef scratch = new BytesRef(); + for (int i = 0; i < numPoints; i++) + { + getValue(i, scratch); + assert scratch.length == packedValue.length; + System.arraycopy(scratch.bytes, scratch.offset, packedValue, 0, packedBytesLength); + visitor.visit(getDocID(i), packedValue); + } + } + + @Override + public int getNumDimensions() + { + return pointDimensionCount; + } + + @Override + public int getBytesPerDimension() + { + return pointNumBytes; + } + + @Override + public long size() + { + return numPoints; + } + + @Override + public int getDocCount() + { + return numRows; + } + }; + } + + private void ensureOpen() + { + Preconditions.checkState(!closed, "Expected open buffer."); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/BKDWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDWriter.java new file mode 100644 index 000000000000..633457747fd6 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/BKDWriter.java @@ -0,0 +1,1042 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.function.IntFunction; + +import com.google.common.base.MoreObjects; + +import org.apache.cassandra.index.sai.disk.MutableOneDimPointValues; +import org.apache.cassandra.index.sai.disk.io.CryptoUtils; +import org.apache.cassandra.index.sai.disk.io.RAMIndexOutput; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.io.compress.ICompressor; +import org.apache.lucene.codecs.MutablePointValues; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.GrowableByteArrayDataOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RAMOutputStream; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.FutureArrays; +import org.apache.lucene.util.IntroSorter; +import org.apache.lucene.util.LongBitSet; +import org.apache.lucene.util.Sorter; +import org.apache.lucene.util.bkd.MutablePointsReaderUtils; + +// TODO +// - allow variable length byte[] (across docs and dims), but this is quite a bit more hairy +// - we could also index "auto-prefix terms" here, and use better compression, and maybe only use for the "fully contained" case so we'd +// only index docIDs +// - the index could be efficiently encoded as an FST, so we don't have wasteful +// (monotonic) long[] leafBlockFPs; or we could use MonotonicLongValues ... but then +// the index is already plenty small: 60M OSM points --> 1.1 MB with 128 points +// per leaf, and you can reduce that by putting more points per leaf +// - we could use threads while building; the higher nodes are very parallelizable + +/** + * Recursively builds a block KD-tree to assign all incoming points in N-dim space to smaller + * and smaller N-dim rectangles (cells) until the number of points in a given + * rectangle is <= maxPointsInLeafNode. The tree is + * fully balanced, which means the leaf nodes will have between 50% and 100% of + * the requested maxPointsInLeafNode. Values that fall exactly + * on a cell boundary may be in either cell. + * + *

    The number of dimensions can be 1 to 8, but every byte[] value is fixed length. + * + *

    + * See this paper for details. + * + *

    This consumes heap during writing: it allocates a LongBitSet(numPoints), + * and then uses up to the specified {@code maxMBSortInHeap} heap space for writing. + * + *

    + * NOTE: This can write at most Integer.MAX_VALUE * maxPointsInLeafNode total points. + * + * @lucene.experimental + */ + +public class BKDWriter implements Closeable +{ + /** How many bytes each docs takes in the fixed-width offline format */ + private final int bytesPerDoc; + + /** Default maximum number of point in each leaf block */ + public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 1024; + + /** Default maximum heap to use, before spilling to (slower) disk */ + public static final float DEFAULT_MAX_MB_SORT_IN_HEAP = 16.0f; + + /** Maximum number of dimensions */ + public static final int MAX_DIMS = 8; + + /** How many dimensions we are indexing */ + protected final int numDims; + + /** How many bytes each value in each dimension takes. */ + protected final int bytesPerDim; + + /** numDims * bytesPerDim */ + protected final int packedBytesLength; + + final BytesRef scratchBytesRef1 = new BytesRef(); + final int[] commonPrefixLengths; + + protected final LongBitSet docsSeen; + + protected final int maxPointsInLeafNode; + private final int maxPointsSortInHeap; + + /** Minimum per-dim values, packed */ + protected final byte[] minPackedValue; + + /** Maximum per-dim values, packed */ + protected final byte[] maxPackedValue; + + protected long pointCount; + + /** true if we have so many values that we must write ords using long (8 bytes) instead of int (4 bytes) */ + protected final boolean longOrds; + + /** An upper bound on how many points the caller will add (includes deletions) */ + private final long totalPointCount; + + private final long maxDoc; + + private final ICompressor compressor; + + public BKDWriter(long maxDoc, int numDims, int bytesPerDim, + int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount, boolean singleValuePerDoc, + ICompressor compressor) throws IOException + { + this(maxDoc, numDims, bytesPerDim, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount, singleValuePerDoc, + totalPointCount > Integer.MAX_VALUE, compressor); + } + + protected BKDWriter(long maxDoc, int numDims, int bytesPerDim, + int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount, + boolean singleValuePerDoc, boolean longOrds, ICompressor compressor) throws IOException + { + verifyParams(numDims, maxPointsInLeafNode, maxMBSortInHeap, totalPointCount); + // We use tracking dir to deal with removing files on exception, so each place that + // creates temp files doesn't need crazy try/finally/sucess logic: + this.maxPointsInLeafNode = maxPointsInLeafNode; + this.numDims = numDims; + this.bytesPerDim = bytesPerDim; + this.totalPointCount = totalPointCount; + this.maxDoc = maxDoc; + this.compressor = compressor; + docsSeen = new LongBitSet(maxDoc); + packedBytesLength = numDims * bytesPerDim; + + commonPrefixLengths = new int[numDims]; + + minPackedValue = new byte[packedBytesLength]; + maxPackedValue = new byte[packedBytesLength]; + + // If we may have more than 1+Integer.MAX_VALUE values, then we must encode ords with long (8 bytes), else we can use int (4 bytes). + this.longOrds = longOrds; + + // dimensional values (numDims * bytesPerDim) + ord (int or long) + docID (int) + if (singleValuePerDoc) + { + // Lucene only supports up to 2.1 docs, so we better not need longOrds in this case: + assert longOrds == false; + bytesPerDoc = packedBytesLength + Integer.BYTES; + } + else if (longOrds) + { + bytesPerDoc = packedBytesLength + Long.BYTES + Integer.BYTES; + } + else + { + bytesPerDoc = packedBytesLength + Integer.BYTES + Integer.BYTES; + } + + // As we recurse, we compute temporary partitions of the data, halving the + // number of points at each recursion. Once there are few enough points, + // we can switch to sorting in heap instead of offline (on disk). At any + // time in the recursion, we hold the number of points at that level, plus + // all recursive halves (i.e. 16 + 8 + 4 + 2) so the memory usage is 2X + // what that level would consume, so we multiply by 0.5 to convert from + // bytes to points here. Each dimension has its own sorted partition, so + // we must divide by numDims as wel. + + maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / (bytesPerDoc * numDims)); + + // Finally, we must be able to hold at least the leaf node in heap during build: + if (maxPointsSortInHeap < maxPointsInLeafNode) + { + throw new IllegalArgumentException("maxMBSortInHeap=" + maxMBSortInHeap + " only allows for maxPointsSortInHeap=" + maxPointsSortInHeap + ", but this is less than maxPointsInLeafNode=" + maxPointsInLeafNode + "; either increase maxMBSortInHeap or decrease maxPointsInLeafNode"); + } + } + + public static void verifyParams(int numDims, int maxPointsInLeafNode, double maxMBSortInHeap, long totalPointCount) + { + // We encode dim in a single byte in the splitPackedValues, but we only expose 4 bits for it now, in case we want to use + // remaining 4 bits for another purpose later + if (numDims < 1 || numDims > MAX_DIMS) + { + throw new IllegalArgumentException("numDims must be 1 .. " + MAX_DIMS + " (got: " + numDims + ")"); + } + if (maxPointsInLeafNode <= 0) + { + throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode); + } + if (maxPointsInLeafNode > ArrayUtil.MAX_ARRAY_LENGTH) + { + throw new IllegalArgumentException("maxPointsInLeafNode must be <= ArrayUtil.MAX_ARRAY_LENGTH (= " + ArrayUtil.MAX_ARRAY_LENGTH + "); got " + maxPointsInLeafNode); + } + if (maxMBSortInHeap < 0.0) + { + throw new IllegalArgumentException("maxMBSortInHeap must be >= 0.0 (got: " + maxMBSortInHeap + ")"); + } + if (totalPointCount < 0) + { + throw new IllegalArgumentException("totalPointCount must be >=0 (got: " + totalPointCount + ")"); + } + } + + /** How many points have been added so far */ + public long getPointCount() + { + return pointCount; + } + + /** + * Write a field from a {@link MutablePointValues}. This way of writing + * points is faster than regular writes with BKDWriter#add since + * there is opportunity for reordering points before writing them to + * disk. This method does not use transient disk in order to reorder points. + */ + public long writeField(IndexOutput out, MutableOneDimPointValues reader, + final OneDimensionBKDWriterCallback callback) throws IOException + { + if (numDims == 1) + { + SAICodecUtils.writeHeader(out); + final long fp = writeField1Dim(out, reader, callback); + SAICodecUtils.writeFooter(out); + return fp; + } + else + { + throw new IllegalArgumentException("Only 1 dimension is supported."); + } + } + + /* In the 1D case, we can simply sort points in ascending order and use the + * same writing logic as we use at merge time. */ + private long writeField1Dim(IndexOutput out, MutableOneDimPointValues reader, + OneDimensionBKDWriterCallback callback) throws IOException + { + // TODO: cast to int + if (reader.size() > 1) + MutablePointsReaderUtils.sort(Math.toIntExact(maxDoc), packedBytesLength, reader, 0, Math.toIntExact(reader.size())); + + final OneDimensionBKDWriter oneDimWriter = new OneDimensionBKDWriter(out, callback); + + reader.intersect((docID, packedValue) -> oneDimWriter.add(packedValue, docID)); + + return oneDimWriter.finish(); + } + + // reused when writing leaf blocks + private final GrowableByteArrayDataOutput scratchOut = new GrowableByteArrayDataOutput(32 * 1024); + + private final GrowableByteArrayDataOutput scratchOut2 = new GrowableByteArrayDataOutput(2 * 1024); + + interface OneDimensionBKDWriterCallback + { + void writeLeafDocs(int leafNum, RowIDAndIndex[] leafDocs, int offset, int count); + } + + public static class RowIDAndIndex + { + public int valueOrderIndex; + public long rowID; + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("valueOrderIndex", valueOrderIndex) + .add("rowID", rowID) + .toString(); + } + } + + private class OneDimensionBKDWriter + { + + final IndexOutput out; + final List leafBlockFPs = new ArrayList<>(); + final List leafBlockStartValues = new ArrayList<>(); + final byte[] leafValues = new byte[maxPointsInLeafNode * packedBytesLength]; + final long[] leafDocs = new long[maxPointsInLeafNode]; + private long valueCount; + private int leafCount; + final RowIDAndIndex[] rowIDAndIndexes = new RowIDAndIndex[maxPointsInLeafNode]; + final int[] orderIndex = new int[maxPointsInLeafNode]; + final OneDimensionBKDWriterCallback callback; + + { + for (int x = 0; x < rowIDAndIndexes.length; x++) + { + rowIDAndIndexes[x] = new RowIDAndIndex(); + } + } + + OneDimensionBKDWriter(IndexOutput out, OneDimensionBKDWriterCallback callback) + { + if (numDims != 1) + { + throw new UnsupportedOperationException("numDims must be 1 but got " + numDims); + } + if (pointCount != 0) + { + throw new IllegalStateException("cannot mix add and merge"); + } + + this.out = out; + this.callback = callback; + + lastPackedValue = new byte[packedBytesLength]; + } + + // for asserts + final byte[] lastPackedValue; + private long lastDocID; + + void add(byte[] packedValue, long docID) throws IOException + { + assert valueInOrder(valueCount + leafCount, + 0, lastPackedValue, packedValue, 0, docID, lastDocID); + + System.arraycopy(packedValue, 0, leafValues, leafCount * packedBytesLength, packedBytesLength); + leafDocs[leafCount] = docID; + docsSeen.set(docID); + leafCount++; + + if (valueCount > totalPointCount) + { + throw new IllegalStateException("totalPointCount=" + totalPointCount + " was passed when we were created, but we just hit " + pointCount + " values"); + } + + if (leafCount == maxPointsInLeafNode) + { + // We write a block once we hit exactly the max count ... this is different from + // when we write N > 1 dimensional points where we write between max/2 and max per leaf block + writeLeafBlock(); + leafCount = 0; + } + + assert (lastDocID = docID) >= 0; // only assign when asserts are enabled + } + + public long finish() throws IOException + { + if (leafCount > 0) + { + writeLeafBlock(); + leafCount = 0; + } + + if (valueCount == 0) + { + return -1; + } + + pointCount = valueCount; + + long indexFP = out.getFilePointer(); + + int numInnerNodes = leafBlockStartValues.size(); + + //System.out.println("BKDW: now rotate numInnerNodes=" + numInnerNodes + " leafBlockStarts=" + leafBlockStartValues.size()); + + byte[] index = new byte[(1 + numInnerNodes) * (1 + bytesPerDim)]; + rotateToTree(1, 0, numInnerNodes, index, leafBlockStartValues); + long[] arr = new long[leafBlockFPs.size()]; + for (int i = 0; i < leafBlockFPs.size(); i++) + { + arr[i] = leafBlockFPs.get(i); + } + writeIndex(out, maxPointsInLeafNode, arr, index); + return indexFP; + } + + private void writeLeafBlock() throws IOException + { + assert leafCount != 0; + if (valueCount == 0) + { + System.arraycopy(leafValues, 0, minPackedValue, 0, packedBytesLength); + } + System.arraycopy(leafValues, (leafCount - 1) * packedBytesLength, maxPackedValue, 0, packedBytesLength); + + valueCount += leafCount; + + if (leafBlockFPs.size() > 0) + { + // Save the first (minimum) value in each leaf block except the first, to build the split value index in the end: + leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength)); + } + leafBlockFPs.add(out.getFilePointer()); + checkMaxLeafNodeCount(leafBlockFPs.size()); + + // Find per-dim common prefix: + int prefix = bytesPerDim; + int offset = (leafCount - 1) * packedBytesLength; + for (int j = 0; j < bytesPerDim; j++) + { + if (leafValues[j] != leafValues[offset + j]) + { + prefix = j; + break; + } + } + + commonPrefixLengths[0] = prefix; + + assert scratchOut.getPosition() == 0; + + out.writeVInt(leafCount); + + for (int x = 0; x < leafCount; x++) + { + rowIDAndIndexes[x].valueOrderIndex = x; + rowIDAndIndexes[x].rowID = leafDocs[x]; + } + + final Sorter sorter = new IntroSorter() + { + RowIDAndIndex pivot; + + @Override + protected void swap(int i, int j) + { + RowIDAndIndex o = rowIDAndIndexes[i]; + rowIDAndIndexes[i] = rowIDAndIndexes[j]; + rowIDAndIndexes[j] = o; + } + + @Override + protected void setPivot(int i) + { + pivot = rowIDAndIndexes[i]; + } + + @Override + protected int comparePivot(int j) + { + return Long.compare(pivot.rowID, rowIDAndIndexes[j].rowID); + } + }; + + sorter.sort(0, leafCount); + + // write leaf rowID -> orig index + scratchOut2.reset(); + + // iterate in row ID order to get the row ID index for the given value order index + // place into an array to be written as packed ints + for (int x = 0; x < leafCount; x++) + { + final int valueOrderIndex = rowIDAndIndexes[x].valueOrderIndex; + orderIndex[valueOrderIndex] = x; + } + + LeafOrderMap.write(orderIndex, leafCount, maxPointsInLeafNode - 1, scratchOut2); + + out.writeVInt(scratchOut2.getPosition()); + out.writeBytes(scratchOut2.getBytes(), 0, scratchOut2.getPosition()); + + if (callback != null) callback.writeLeafDocs(leafBlockFPs.size() - 1, rowIDAndIndexes, 0, leafCount); + + writeCommonPrefixes(scratchOut, commonPrefixLengths, leafValues); + + scratchBytesRef1.length = packedBytesLength; + scratchBytesRef1.bytes = leafValues; + + final IntFunction packedValues = (i) -> { + scratchBytesRef1.offset = packedBytesLength * i; + return scratchBytesRef1; + }; + assert valuesInOrderAndBounds(leafCount, 0, ArrayUtil.copyOfSubArray(leafValues, 0, packedBytesLength), + ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength), + packedValues, leafDocs, 0); + + writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, leafCount, 0, packedValues); + + if (compressor == null) + { + out.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition()); + } + else + { + CryptoUtils.compress(new BytesRef(scratchOut.getBytes(), 0, scratchOut.getPosition()), scratchBytesRef, out, compressor); + } + scratchOut.reset(); + } + } + + private final BytesRef scratchBytesRef = new BytesRef(new byte[128]); + + // TODO: there must be a simpler way? + private void rotateToTree(int nodeID, int offset, int count, byte[] index, List leafBlockStartValues) + { + //System.out.println("ROTATE: nodeID=" + nodeID + " offset=" + offset + " count=" + count + " bpd=" + bytesPerDim + " index.length=" + index.length); + if (count == 1) + { + // Leaf index node + //System.out.println(" leaf index node"); + //System.out.println(" index[" + nodeID + "] = blockStartValues[" + offset + "]"); + System.arraycopy(leafBlockStartValues.get(offset), 0, index, nodeID * (1 + bytesPerDim) + 1, bytesPerDim); + } + else if (count > 1) + { + // Internal index node: binary partition of count + int countAtLevel = 1; + int totalCount = 0; + while (true) + { + int countLeft = count - totalCount; + //System.out.println(" cycle countLeft=" + countLeft + " coutAtLevel=" + countAtLevel); + if (countLeft <= countAtLevel) + { + // This is the last level, possibly partially filled: + int lastLeftCount = Math.min(countAtLevel / 2, countLeft); + assert lastLeftCount >= 0; + int leftHalf = (totalCount - 1) / 2 + lastLeftCount; + + int rootOffset = offset + leftHalf; + /* + System.out.println(" last left count " + lastLeftCount); + System.out.println(" leftHalf " + leftHalf + " rightHalf=" + (count-leftHalf-1)); + System.out.println(" rootOffset=" + rootOffset); + */ + + System.arraycopy(leafBlockStartValues.get(rootOffset), 0, index, nodeID * (1 + bytesPerDim) + 1, bytesPerDim); + //System.out.println(" index[" + nodeID + "] = blockStartValues[" + rootOffset + "]"); + + // TODO: we could optimize/specialize, when we know it's simply fully balanced binary tree + // under here, to save this while loop on each recursion + + // Recurse left + rotateToTree(2 * nodeID, offset, leftHalf, index, leafBlockStartValues); + + // Recurse right + rotateToTree(2 * nodeID + 1, rootOffset + 1, count - leftHalf - 1, index, leafBlockStartValues); + return; + } + totalCount += countAtLevel; + countAtLevel *= 2; + } + } + else + { + assert count == 0; + } + } + + // useful for debugging: + /* + private void printPathSlice(String desc, PathSlice slice, int dim) throws IOException { + System.out.println(" " + desc + " dim=" + dim + " count=" + slice.count + ":"); + try(PointReader r = slice.writer.getReader(slice.start, slice.count)) { + int count = 0; + while (r.next()) { + byte[] v = r.packedValue(); + System.out.println(" " + count + ": " + new BytesRef(v, dim*bytesPerDim, bytesPerDim)); + count++; + if (count == slice.count) { + break; + } + } + } + } + */ + + private void checkMaxLeafNodeCount(int numLeaves) + { + if ((1 + bytesPerDim) * (long) numLeaves > ArrayUtil.MAX_ARRAY_LENGTH) + { + throw new IllegalStateException("too many nodes; increase maxPointsInLeafNode (currently " + maxPointsInLeafNode + ") and reindex"); + } + } + + /** Packs the two arrays, representing a balanced binary tree, into a compact byte[] structure. */ + @SuppressWarnings("resource") + private byte[] packIndex(long[] leafBlockFPs, byte[] splitPackedValues) throws IOException + { + + int numLeaves = leafBlockFPs.length; + + // Possibly rotate the leaf block FPs, if the index not fully balanced binary tree (only happens + // if it was created by OneDimensionBKDWriter). In this case the leaf nodes may straddle the two bottom + // levels of the binary tree: + if (numDims == 1 && numLeaves > 1) + { + int levelCount = 2; + while (true) + { + if (numLeaves >= levelCount && numLeaves <= 2 * levelCount) + { + int lastLevel = 2 * (numLeaves - levelCount); + assert lastLevel >= 0; + if (lastLevel != 0) + { + // Last level is partially filled, so we must rotate the leaf FPs to match. We do this here, after loading + // at read-time, so that we can still delta code them on disk at write: + long[] newLeafBlockFPs = new long[numLeaves]; + System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel); + System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel); + leafBlockFPs = newLeafBlockFPs; + } + break; + } + + levelCount *= 2; + } + } + + /** Reused while packing the index */ + // TODO: replace with RAMIndexOutput because RAMOutputStream has synchronized/monitor locks + RAMOutputStream writeBuffer = new RAMOutputStream(); + + // This is the "file" we append the byte[] to: + List blocks = new ArrayList<>(); + byte[] lastSplitValues = new byte[bytesPerDim * numDims]; + //System.out.println("\npack index"); + int totalSize = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, 0l, blocks, 1, lastSplitValues, new boolean[numDims], false); + + // Compact the byte[] blocks into single byte index: + byte[] index = new byte[totalSize]; + int upto = 0; + for (byte[] block : blocks) + { + System.arraycopy(block, 0, index, upto, block.length); + upto += block.length; + } + assert upto == totalSize; + + return index; + } + + /** Appends the current contents of writeBuffer as another block on the growing in-memory file */ + private int appendBlock(RAMOutputStream writeBuffer, List blocks) throws IOException + { + int pos = Math.toIntExact(writeBuffer.getFilePointer()); + byte[] bytes = new byte[pos]; + writeBuffer.writeTo(bytes, 0); + writeBuffer.reset(); + blocks.add(bytes); + return pos; + } + + /** + * lastSplitValues is per-dimension split value previously seen; we use this to prefix-code the split byte[] on each + * inner node + */ + private int recursePackIndex(RAMOutputStream writeBuffer, long[] leafBlockFPs, byte[] splitPackedValues, long minBlockFP, List blocks, + int nodeID, byte[] lastSplitValues, boolean[] negativeDeltas, boolean isLeft) throws IOException + { + if (nodeID >= leafBlockFPs.length) + { + int leafID = nodeID - leafBlockFPs.length; + //System.out.println("recursePack leaf nodeID=" + nodeID); + + // In the unbalanced case it's possible the left most node only has one child: + if (leafID < leafBlockFPs.length) + { + long delta = leafBlockFPs[leafID] - minBlockFP; + if (isLeft) + { + assert delta == 0; + return 0; + } + else + { + assert nodeID == 1 || delta > 0 : "nodeID=" + nodeID; + writeBuffer.writeVLong(delta); + return appendBlock(writeBuffer, blocks); + } + } + else + { + return 0; + } + } + else + { + long leftBlockFP; + if (isLeft == false) + { + leftBlockFP = getLeftMostLeafBlockFP(leafBlockFPs, nodeID); + long delta = leftBlockFP - minBlockFP; + assert nodeID == 1 || delta > 0; + writeBuffer.writeVLong(delta); + } + else + { + // The left tree's left most leaf block FP is always the minimal FP: + leftBlockFP = minBlockFP; + } + + int address = nodeID * (1 + bytesPerDim); + int splitDim = splitPackedValues[address++] & 0xff; + + //System.out.println("recursePack inner nodeID=" + nodeID + " splitDim=" + splitDim + " splitValue=" + new BytesRef(splitPackedValues, address, bytesPerDim)); + + // find common prefix with last split value in this dim: + int prefix = 0; + for (; prefix < bytesPerDim; prefix++) + { + if (splitPackedValues[address + prefix] != lastSplitValues[splitDim * bytesPerDim + prefix]) + { + break; + } + } + + //System.out.println("writeNodeData nodeID=" + nodeID + " splitDim=" + splitDim + " numDims=" + numDims + " bytesPerDim=" + bytesPerDim + " prefix=" + prefix); + + int firstDiffByteDelta; + if (prefix < bytesPerDim) + { + //System.out.println(" delta byte cur=" + Integer.toHexString(splitPackedValues[address+prefix]&0xFF) + " prev=" + Integer.toHexString(lastSplitValues[splitDim * bytesPerDim + prefix]&0xFF) + " negated?=" + negativeDeltas[splitDim]); + firstDiffByteDelta = (splitPackedValues[address + prefix] & 0xFF) - (lastSplitValues[splitDim * bytesPerDim + prefix] & 0xFF); + if (negativeDeltas[splitDim]) + { + firstDiffByteDelta = -firstDiffByteDelta; + } + //System.out.println(" delta=" + firstDiffByteDelta); + assert firstDiffByteDelta > 0; + } + else + { + firstDiffByteDelta = 0; + } + + // pack the prefix, splitDim and delta first diff byte into a single vInt: + int code = (firstDiffByteDelta * (1 + bytesPerDim) + prefix) * numDims + splitDim; + + //System.out.println(" code=" + code); + //System.out.println(" splitValue=" + new BytesRef(splitPackedValues, address, bytesPerDim)); + + writeBuffer.writeVInt(code); + + // write the split value, prefix coded vs. our parent's split value: + int suffix = bytesPerDim - prefix; + byte[] savSplitValue = new byte[suffix]; + if (suffix > 1) + { + writeBuffer.writeBytes(splitPackedValues, address + prefix + 1, suffix - 1); + } + + byte[] cmp = lastSplitValues.clone(); + + System.arraycopy(lastSplitValues, splitDim * bytesPerDim + prefix, savSplitValue, 0, suffix); + + // copy our split value into lastSplitValues for our children to prefix-code against + System.arraycopy(splitPackedValues, address + prefix, lastSplitValues, splitDim * bytesPerDim + prefix, suffix); + + int numBytes = appendBlock(writeBuffer, blocks); + + // placeholder for left-tree numBytes; we need this so that at search time if we only need to recurse into the right sub-tree we can + // quickly seek to its starting point + int idxSav = blocks.size(); + blocks.add(null); + + boolean savNegativeDelta = negativeDeltas[splitDim]; + negativeDeltas[splitDim] = true; + + int leftNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, leftBlockFP, blocks, 2 * nodeID, lastSplitValues, negativeDeltas, true); + + if (nodeID * 2 < leafBlockFPs.length) + { + writeBuffer.writeVInt(leftNumBytes); + } + else + { + assert leftNumBytes == 0 : "leftNumBytes=" + leftNumBytes; + } + int numBytes2 = Math.toIntExact(writeBuffer.getFilePointer()); + byte[] bytes2 = new byte[numBytes2]; + writeBuffer.writeTo(bytes2, 0); + writeBuffer.reset(); + // replace our placeholder: + blocks.set(idxSav, bytes2); + + negativeDeltas[splitDim] = false; + int rightNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitPackedValues, leftBlockFP, blocks, 2 * nodeID + 1, lastSplitValues, negativeDeltas, false); + + negativeDeltas[splitDim] = savNegativeDelta; + + // restore lastSplitValues to what caller originally passed us: + System.arraycopy(savSplitValue, 0, lastSplitValues, splitDim * bytesPerDim + prefix, suffix); + + assert Arrays.equals(lastSplitValues, cmp); + + return numBytes + numBytes2 + leftNumBytes + rightNumBytes; + } + } + + private long getLeftMostLeafBlockFP(long[] leafBlockFPs, int nodeID) + { + // TODO: can we do this cheaper, e.g. a closed form solution instead of while loop? Or + // change the recursion while packing the index to return this left-most leaf block FP + // from each recursion instead? + // + // Still, the overall cost here is minor: this method's cost is O(log(N)), and while writing + // we call it O(N) times (N = number of leaf blocks) + while (nodeID < leafBlockFPs.length) + { + nodeID *= 2; + } + int leafID = nodeID - leafBlockFPs.length; + long result = leafBlockFPs[leafID]; + if (result < 0) + { + throw new AssertionError(result + " for leaf " + leafID); + } + return result; + } + + private void writeIndex(IndexOutput out, int countPerLeaf, long[] leafBlockFPs, byte[] splitPackedValues) throws IOException + { + byte[] packedIndex = packIndex(leafBlockFPs, splitPackedValues); + writeIndex(out, countPerLeaf, leafBlockFPs.length, packedIndex); + } + + private void writeIndex(IndexOutput out, int countPerLeaf, int numLeaves, byte[] packedIndex) throws IOException + { + out.writeVInt(numDims); + out.writeVInt(countPerLeaf); + out.writeVInt(bytesPerDim); + + assert numLeaves > 0; + out.writeVInt(numLeaves); + + if (compressor != null) + { + RAMIndexOutput ramOut = new RAMIndexOutput(""); + ramOut.writeBytes(minPackedValue, 0, packedBytesLength); + ramOut.writeBytes(maxPackedValue, 0, packedBytesLength); + + CryptoUtils.compress(new BytesRef(ramOut.getBytes(), 0, (int)ramOut.getFilePointer()), out, compressor); + } + else + { + out.writeBytes(minPackedValue, 0, packedBytesLength); + out.writeBytes(maxPackedValue, 0, packedBytesLength); + } + + out.writeVLong(pointCount); + //TODO Changing disk format + out.writeVLong(docsSeen.cardinality()); + + if (compressor != null) + { + CryptoUtils.compress(new BytesRef(packedIndex, 0, packedIndex.length), out, compressor); + } + else + { + out.writeVInt(packedIndex.length); + out.writeBytes(packedIndex, 0, packedIndex.length); + } + } + + private void writeLeafBlockPackedValues(DataOutput out, int[] commonPrefixLengths, int count, int sortedDim, IntFunction packedValues) throws IOException + { + int prefixLenSum = Arrays.stream(commonPrefixLengths).sum(); + if (prefixLenSum == packedBytesLength) + { + // all values in this block are equal + out.writeByte((byte) -1); + } + else + { + assert numDims == 1; + + assert commonPrefixLengths[sortedDim] < bytesPerDim; + out.writeByte((byte) sortedDim); + int compressedByteOffset = sortedDim * bytesPerDim + commonPrefixLengths[sortedDim]; + commonPrefixLengths[sortedDim]++; + for (int i = 0; i < count; ) + { + // do run-length compression on the byte at compressedByteOffset + int runLen = runLen(packedValues, i, Math.min(i + 0xff, count), compressedByteOffset); + assert runLen <= 0xff; + BytesRef first = packedValues.apply(i); + byte prefixByte = first.bytes[first.offset + compressedByteOffset]; + out.writeByte(prefixByte); + out.writeByte((byte) runLen); + writeLeafBlockPackedValuesRange(out, commonPrefixLengths, i, i + runLen, packedValues); + i += runLen; + assert i <= count; + } + } + } + + /** + * Return an array that contains the min and max values for the [offset, offset+length] interval + * of the given {@link BytesRef}s. + */ + private static BytesRef[] computeMinMax(int count, IntFunction packedValues, int offset, int length) + { + assert length > 0; + BytesRefBuilder min = new BytesRefBuilder(); + BytesRefBuilder max = new BytesRefBuilder(); + BytesRef first = packedValues.apply(0); + min.copyBytes(first.bytes, first.offset + offset, length); + max.copyBytes(first.bytes, first.offset + offset, length); + for (int i = 1; i < count; ++i) + { + BytesRef candidate = packedValues.apply(i); + if (FutureArrays.compareUnsigned(min.bytes(), 0, length, candidate.bytes, candidate.offset + offset, candidate.offset + offset + length) > 0) + { + min.copyBytes(candidate.bytes, candidate.offset + offset, length); + } + else if (FutureArrays.compareUnsigned(max.bytes(), 0, length, candidate.bytes, candidate.offset + offset, candidate.offset + offset + length) < 0) + { + max.copyBytes(candidate.bytes, candidate.offset + offset, length); + } + } + return new BytesRef[]{ min.get(), max.get() }; + } + + private void writeLeafBlockPackedValuesRange(DataOutput out, int[] commonPrefixLengths, int start, int end, IntFunction packedValues) throws IOException + { + for (int i = start; i < end; ++i) + { + BytesRef ref = packedValues.apply(i); + assert ref.length == packedBytesLength; + + for (int dim = 0; dim < numDims; dim++) + { + int prefix = commonPrefixLengths[dim]; + out.writeBytes(ref.bytes, ref.offset + dim * bytesPerDim + prefix, bytesPerDim - prefix); + } + } + } + + private static int runLen(IntFunction packedValues, int start, int end, int byteOffset) + { + BytesRef first = packedValues.apply(start); + byte b = first.bytes[first.offset + byteOffset]; + for (int i = start + 1; i < end; ++i) + { + BytesRef ref = packedValues.apply(i); + byte b2 = ref.bytes[ref.offset + byteOffset]; + assert Byte.toUnsignedInt(b2) >= Byte.toUnsignedInt(b); + if (b != b2) + { + return i - start; + } + } + return end - start; + } + + private void writeCommonPrefixes(DataOutput out, int[] commonPrefixes, byte[] packedValue) throws IOException + { + for (int dim = 0; dim < numDims; dim++) + { + out.writeVInt(commonPrefixes[dim]); + //System.out.println(commonPrefixes[dim] + " of " + bytesPerDim); + out.writeBytes(packedValue, dim * bytesPerDim, commonPrefixes[dim]); + } + } + + @Override + public void close() throws IOException + { + + } + + /** Called only in assert */ + private boolean valueInBounds(BytesRef packedValue, byte[] minPackedValue, byte[] maxPackedValue) + { + for (int dim = 0; dim < numDims; dim++) + { + int offset = bytesPerDim * dim; + if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDim, minPackedValue, offset, offset + bytesPerDim) < 0) + { + return false; + } + if (FutureArrays.compareUnsigned(packedValue.bytes, packedValue.offset + offset, packedValue.offset + offset + bytesPerDim, maxPackedValue, offset, offset + bytesPerDim) > 0) + { + return false; + } + } + + return true; + } + + // only called from assert + private boolean valuesInOrderAndBounds(int count, int sortedDim, byte[] minPackedValue, byte[] maxPackedValue, + IntFunction values, long[] docs, int docsOffset) throws IOException + { + byte[] lastPackedValue = new byte[packedBytesLength]; + long lastDoc = -1; + for (int i = 0; i < count; i++) + { + BytesRef packedValue = values.apply(i); + assert packedValue.length == packedBytesLength; + assert valueInOrder(i, sortedDim, lastPackedValue, packedValue.bytes, packedValue.offset, + docs[docsOffset + i], lastDoc); + lastDoc = docs[docsOffset + i]; + + // Make sure this value does in fact fall within this leaf cell: + assert valueInBounds(packedValue, minPackedValue, maxPackedValue); + } + return true; + } + + // only called from assert + private boolean valueInOrder(long ord, int sortedDim, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset, + long doc, long lastDoc) + { + int dimOffset = sortedDim * bytesPerDim; + if (ord > 0) + { + int cmp = FutureArrays.compareUnsigned(lastPackedValue, dimOffset, dimOffset + bytesPerDim, packedValue, packedValueOffset + dimOffset, packedValueOffset + dimOffset + bytesPerDim); + if (cmp > 0) + { + throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) + " current value=" + new BytesRef(packedValue, packedValueOffset, packedBytesLength) + " ord=" + ord); + } + if (cmp == 0 && doc < lastDoc) + { + throw new AssertionError("docs out of order: last doc=" + lastDoc + " current doc=" + doc + " ord=" + ord); + } + } + System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, packedBytesLength); + return true; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/BlockPackedReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/BlockPackedReader.java new file mode 100644 index 000000000000..17572e597b3c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/BlockPackedReader.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.index.sai.disk.v1; + +import java.io.IOException; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.io.IndexInputReader; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; + +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.checkBlockSize; +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.numBlocks; +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.readVLong; +import static org.apache.lucene.util.BitUtil.zigZagDecode; + +/** + * Provides non-blocking, random access to a stream written with {@link BlockPackedWriter}. + */ +public class BlockPackedReader implements LongArray.Factory +{ + private final IndexComponents components; + private final FileHandle file; + private final int blockShift, blockMask; + private final long valueCount; + private final byte[] blockBitsPerValue; + private final long[] blockOffsets; + private final long[] minValues; + + public BlockPackedReader(FileHandle file, Component component, IndexComponents components, MetadataSource source) throws IOException + { + this(file, components, new NumericValuesMeta(source.get(component.name()))); + } + + @SuppressWarnings("resource") + public BlockPackedReader(FileHandle file, IndexComponents components, NumericValuesMeta meta) throws IOException + { + this.components = components; + this.file = file; + + this.valueCount = meta.valueCount; + + blockShift = checkBlockSize(meta.blockSize, AbstractBlockPackedWriter.MIN_BLOCK_SIZE, AbstractBlockPackedWriter.MAX_BLOCK_SIZE); + blockMask = meta.blockSize - 1; + final int numBlocks = numBlocks(valueCount, meta.blockSize); + blockBitsPerValue = new byte[numBlocks]; + blockOffsets = new long[numBlocks]; + minValues = new long[numBlocks]; + + try (final RandomAccessReader reader = this.file.createReader()) + { + final IndexInputReader in = IndexInputReader.create(reader); + SAICodecUtils.validate(in); + in.seek(meta.blockMetaOffset); + + for (int i = 0; i < numBlocks; ++i) + { + final int token = in.readByte() & 0xFF; + final int bitsPerValue = token >>> AbstractBlockPackedWriter.BPV_SHIFT; + if (bitsPerValue > 64) + { + throw new CorruptIndexException(String.format("Block %d is corrupted. Bits per value should be no more than 64 and is %d.", i, bitsPerValue), in); + } + if ((token & AbstractBlockPackedWriter.MIN_VALUE_EQUALS_0) == 0) + { + long val = zigZagDecode(1L + readVLong(in)); + minValues[i] = val; + } + else + { + minValues[i] = 0L; + } + + blockBitsPerValue[i] = (byte) bitsPerValue; + + if (bitsPerValue > 0) + { + blockOffsets[i] = in.readVLong(); + } + else + { + blockOffsets[i] = -1; + } + } + } + } + + @VisibleForTesting + @Override + public LongArray open() + { + return openTokenReader(0, null); + } + + @Override + @SuppressWarnings("resource") + public LongArray openTokenReader(long sstableRowId, SSTableQueryContext context) + { + final IndexInput indexInput = components.openInput(file); + return new AbstractBlockPackedReader(indexInput, blockBitsPerValue, blockShift, blockMask, sstableRowId, valueCount) + { + @Override + protected long blockOffsetAt(int block) + { + return blockOffsets[block]; + } + + @Override + long delta(int block, int idx) + { + return minValues[block]; + } + + @Override + public void close() throws IOException + { + indexInput.close(); + } + }; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/BlockPackedWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/BlockPackedWriter.java new file mode 100644 index 000000000000..2309d4bb9286 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/BlockPackedWriter.java @@ -0,0 +1,79 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.packed.DirectWriter; + +import static org.apache.lucene.util.BitUtil.zigZagEncode; + +/** + * A writer for large sequences of longs. + * + * Modified copy of {@link org.apache.lucene.util.packed.BlockPackedWriter} to use {@link DirectWriter} + * for optimised reads that doesn't require seeking through the whole file to open a thread-exclusive reader. + */ +public class BlockPackedWriter extends AbstractBlockPackedWriter +{ + public BlockPackedWriter(IndexOutput out, int blockSize) + { + super(out, blockSize); + } + + @Override + protected void flush() throws IOException + { + assert off > 0; + long min = Long.MAX_VALUE, max = Long.MIN_VALUE; + for (int i = 0; i < off; ++i) + { + min = Math.min(values[i], min); + max = Math.max(values[i], max); + } + + final long delta = max - min; + int bitsRequired = delta == 0 ? 0 : DirectWriter.unsignedBitsRequired(delta); + + final int token = (bitsRequired << BPV_SHIFT) | (min == 0 ? MIN_VALUE_EQUALS_0 : 0); + blockMetaWriter.writeByte((byte) token); + + if (min != 0) + { + // TODO: the min values can be delta encoded since they are read linearly + // TODO: buffer the min values so they may be written as a single block + writeVLong(blockMetaWriter, zigZagEncode(min) - 1); + } + + if (bitsRequired > 0) + { + if (min != 0) + { + for (int i = 0; i < off; ++i) + { + values[i] -= min; + } + } + blockMetaWriter.writeVLong(out.getFilePointer()); + writeValues(off, bitsRequired); + } + + off = 0; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/DirectReaders.java b/src/java/org/apache/cassandra/index/sai/disk/v1/DirectReaders.java new file mode 100644 index 000000000000..420348f62ab5 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/DirectReaders.java @@ -0,0 +1,235 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.lucene.store.RandomAccessInput; + +class DirectReaders +{ + interface Reader + { + long get(RandomAccessInput in, long offset, long index); + } + + static Reader getReaderForBitsPerValue(byte bitsPerValue) + { + switch (bitsPerValue) + { + case 0: + return READER_0; + case 1: + return READER_1; + case 2: + return READER_2; + case 4: + return READER_4; + case 8: + return READER_8; + case 12: + return READER_12; + case 16: + return READER_16; + case 20: + return READER_20; + case 24: + return READER_24; + case 28: + return READER_28; + case 32: + return READER_32; + case 40: + return READER_40; + case 48: + return READER_48; + case 56: + return READER_56; + case 64: + return READER_64; + default: + throw new IllegalArgumentException("unsupported bitsPerValue: " + bitsPerValue); + } + } + + private static final Reader READER_0 = (in, offset, index) -> 0; + + private static final Reader READER_1 = (in, offset, index) -> { + try + { + int shift = 7 - (int) (index & 7); + return (in.readByte(offset + (index >>> 3)) >>> shift) & 0x1; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_2 = (in, offset, index) -> { + try + { + int shift = (3 - (int) (index & 3)) << 1; + return (in.readByte(offset + (index >>> 2)) >>> shift) & 0x3; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_4 = (in, offset, index) -> { + try + { + int shift = (int) ((index + 1) & 1) << 2; + return (in.readByte(offset + (index >>> 1)) >>> shift) & 0xF; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_8 = (in, offset, index) -> { + try + { + return in.readByte(offset + index) & 0xFF; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_12 = (in, offset, index) -> { + try + { + long o = (index * 12) >>> 3; + int shift = (int) ((index + 1) & 1) << 2; + return (in.readShort(offset + o) >>> shift) & 0xFFF; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_16 = (in, offset, index) -> { + try + { + return in.readShort(offset + (index << 1)) & 0xFFFF; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_20 = (in, offset, index) -> { + try + { + long o = (index * 20) >>> 3; + int v = in.readInt(offset + o) >>> 8; + int shift = (int) ((index + 1) & 1) << 2; + return (v >>> shift) & 0xFFFFF; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_24 = (in, offset, index) -> { + try + { + return in.readInt(offset + index * 3) >>> 8; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_28 = (in, offset, index) -> { + try + { + long o = (index * 28) >>> 3; + int shift = (int) ((index + 1) & 1) << 2; + return (in.readInt(offset + o) >>> shift) & 0xFFFFFFFL; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_32 = (in, offset, index) -> { + try + { + return in.readInt(offset + (index << 2)) & 0xFFFFFFFFL; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_40 = (in, offset, index) -> { + try + { + return in.readLong(offset + index * 5) >>> 24; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_48 = (in, offset, index) -> { + try + { + return in.readLong(offset + index * 6) >>> 16; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_56 = (in, offset, index) -> { + try + { + return in.readLong(offset + index * 7) >>> 8; + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; + + private static final Reader READER_64 = (in, offset, index) -> { + try + { + return in.readLong(offset + (index << 3)); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }; +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/FilteringPostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/FilteringPostingList.java new file mode 100644 index 000000000000..cabf733dcd5d --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/FilteringPostingList.java @@ -0,0 +1,106 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import com.google.common.base.Preconditions; + +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.lucene.util.FixedBitSet; + + +/** + * A wrapper that iterates over a delegate {@link PostingList}, filtering out postings at + * positions that are not present in a provided filter. + */ +public class FilteringPostingList implements PostingList +{ + private final FixedBitSet filter; + private final OrdinalPostingList delegate; + private final int cardinality; + private int position = 0; + + FilteringPostingList(FixedBitSet filter, OrdinalPostingList delegate) + { + cardinality = filter.cardinality(); + + Preconditions.checkArgument(cardinality > 0, "Filter must contain at least one match."); + + this.filter = filter; + this.delegate = delegate; + } + + @Override + public void close() throws IOException + { + delegate.close(); + } + + /** + * + * @return the segment row ID of the next match + */ + @Override + public long nextPosting() throws IOException + { + while (true) + { + long segmentRowId = delegate.nextPosting(); + + if (segmentRowId == PostingList.END_OF_STREAM) + { + return PostingList.END_OF_STREAM; + } + + if (filter.get(position++)) + { + return segmentRowId; + } + } + } + + @Override + public long size() + { + return cardinality; + } + + @Override + public long advance(long targetRowID) throws IOException + { + long segmentRowId = delegate.advance(targetRowID); + + if (segmentRowId == PostingList.END_OF_STREAM) + { + return PostingList.END_OF_STREAM; + } + + // these are always for leaf kdtree postings so the max is 1024 + position = (int)delegate.getOrdinal(); + + // If the ordinal of the ID we just read satisfies the filter, just return it... + if (filter.get(position - 1)) + { + return segmentRowId; + } + + // ...but if the ID doesn't satisfy the filter, get the next match. + return nextPosting(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/InvertedIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/InvertedIndexWriter.java new file mode 100644 index 000000000000..3d444244e3c4 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/InvertedIndexWriter.java @@ -0,0 +1,109 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.commons.lang3.mutable.MutableLong; + +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.cassandra.index.sai.disk.SegmentMetadata; +import org.apache.cassandra.index.sai.disk.TermsIterator; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * Builds an on-disk inverted index structure: terms dictionary and postings lists. + */ +@NotThreadSafe +public class InvertedIndexWriter implements Closeable +{ + private final TrieTermsDictionaryWriter termsDictionaryWriter; + private final PostingsWriter postingsWriter; + private long postingsAdded; + + public InvertedIndexWriter(IndexComponents indexComponents, boolean segmented) throws IOException + { + this.termsDictionaryWriter = new TrieTermsDictionaryWriter(indexComponents, segmented); + this.postingsWriter = new PostingsWriter(indexComponents, segmented); + } + + /** + * Appends a set of terms and associated postings to their respective overall SSTable component files. + * + * @param terms an iterator of terms with their associated postings + * + * @return metadata describing the location of this inverted index in the overall SSTable + * terms and postings component files + */ + public SegmentMetadata.ComponentMetadataMap writeAll(TermsIterator terms) throws IOException + { + // Terms and postings writers are opened in append mode with pointers at the end of their respective files. + long termsOffset = termsDictionaryWriter.getStartOffset(); + long postingsOffset = postingsWriter.getStartOffset(); + + while (terms.hasNext()) + { + ByteComparable term = terms.next(); + try (PostingList postings = terms.postings()) + { + final long offset = postingsWriter.write(postings); + if (offset >= 0) + termsDictionaryWriter.add(term, offset); + } + } + postingsAdded = postingsWriter.getTotalPostings(); + MutableLong footerPointer = new MutableLong(); + long termsRoot = termsDictionaryWriter.complete(footerPointer); + postingsWriter.complete(); + + long termsLength = termsDictionaryWriter.getFilePointer() - termsOffset; + long postingsLength = postingsWriter.getFilePointer() - postingsOffset; + + SegmentMetadata.ComponentMetadataMap components = new SegmentMetadata.ComponentMetadataMap(); + + Map map = new HashMap<>(2); + map.put(SAICodecUtils.FOOTER_POINTER, "" + footerPointer.getValue()); + + // Postings list file pointers are stored directly in TERMS_DATA, so a root is not needed. + components.put(IndexComponents.NDIType.POSTING_LISTS, -1, postingsOffset, postingsLength); + components.put(IndexComponents.NDIType.TERMS_DATA, termsRoot, termsOffset, termsLength, map); + + return components; + } + + @Override + public void close() throws IOException + { + postingsWriter.close(); + termsDictionaryWriter.close(); + } + + /** + * @return total number of row IDs added to posting lists + */ + public long getPostingsCount() + { + return postingsAdded; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/LeafOrderMap.java b/src/java/org/apache/cassandra/index/sai/disk/v1/LeafOrderMap.java new file mode 100644 index 000000000000..a5ca47809c50 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/LeafOrderMap.java @@ -0,0 +1,45 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.util.packed.DirectWriter; + +public class LeafOrderMap +{ + public static int getValue(RandomAccessInput in, long offset, int index, DirectReaders.Reader reader) + { + return Math.toIntExact(reader.get(in, offset, index)); + } + + public static void write(final int[] array, int length, int maxValue, final DataOutput out) throws IOException + { + final int bits = DirectWriter.unsignedBitsRequired(maxValue); + final DirectWriter writer = DirectWriter.getInstance(out, length, bits); + for (int i = 0; i < length; i++) + { + assert array[i] <= maxValue; + + writer.add(array[i]); + } + writer.finish(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/MergePostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/MergePostingList.java new file mode 100644 index 000000000000..a7acc55f574c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/MergePostingList.java @@ -0,0 +1,126 @@ +/* + * 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.index.sai.disk.v1; + + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.PriorityQueue; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.cassandra.io.util.FileUtils; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * Merges multiple {@link PostingList} which individually contain unique items into a single list. + */ +@NotThreadSafe +public class MergePostingList implements PostingList +{ + final PriorityQueue postingLists; + final List temp; + final Closeable onClose; + final long size; + private long lastRowId = -1; + + private MergePostingList(PriorityQueue postingLists, Closeable onClose) + { + this.temp = new ArrayList<>(postingLists.size()); + this.onClose = onClose; + this.postingLists = postingLists; + long size = 0; + for (PostingList postingList : postingLists) + { + size += postingList.size(); + } + this.size = size; + } + + public static PostingList merge(PriorityQueue postings, Closeable onClose) + { + checkArgument(!postings.isEmpty()); + return postings.size() > 1 ? new MergePostingList(postings, onClose) : postings.poll(); + } + + public static PostingList merge(PriorityQueue postings) + { + return merge(postings, () -> FileUtils.close(postings)); + } + + @SuppressWarnings("resource") + @Override + public long nextPosting() throws IOException + { + while (!postingLists.isEmpty()) + { + PeekablePostingList head = postingLists.poll(); + long next = head.nextPosting(); + + if (next == END_OF_STREAM) + { + // skip current posting list + } + else if (next > lastRowId) + { + lastRowId = next; + postingLists.add(head); + return next; + } + else if (next == lastRowId) + { + postingLists.add(head); + } + } + + return PostingList.END_OF_STREAM; + } + + @SuppressWarnings("resource") + @Override + public long advance(long targetRowID) throws IOException + { + temp.clear(); + + while (!postingLists.isEmpty()) + { + PeekablePostingList peekable = postingLists.poll(); + peekable.advanceWithoutConsuming(targetRowID); + if (peekable.peek() != PostingList.END_OF_STREAM) + temp.add(peekable); + } + postingLists.addAll(temp); + + return nextPosting(); + } + + @Override + public long size() + { + return size; + } + + @Override + public void close() throws IOException + { + onClose.close(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/MetadataSource.java b/src/java/org/apache/cassandra/index/sai/disk/v1/MetadataSource.java new file mode 100644 index 000000000000..766c90865f1c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/MetadataSource.java @@ -0,0 +1,104 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.cassandra.index.sai.disk.format.Version; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.lucene.store.BufferedChecksumIndexInput; +import org.apache.lucene.store.ByteArrayIndexInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; + +@NotThreadSafe +public class MetadataSource +{ + private final Version version; + private final Map components; + + private MetadataSource(Version version, Map components) + { + this.version = version; + this.components = components; + } + + public static MetadataSource loadGroupMetadata(IndexComponents components) throws IOException + { + return MetadataSource.load(components.openBlockingInput(IndexComponents.GROUP_META)); + } + + public static MetadataSource loadColumnMetadata(IndexComponents components) throws IOException + { + return MetadataSource.load(components.openBlockingInput(components.meta)); + } + + private static MetadataSource load(IndexInput indexInput) throws IOException + { + Map components = new HashMap<>(); + Version version; + + try (BufferedChecksumIndexInput input = new BufferedChecksumIndexInput(indexInput)) + { + version = SAICodecUtils.checkHeader(input); + final int num = input.readInt(); + + for (int x = 0; x < num; x++) + { + if (input.length() == input.getFilePointer()) + { + // we should never get here, because we always add footer to the file + throw new IllegalStateException("Unexpected EOF in " + input); + } + + final String name = input.readString(); + final int length = input.readInt(); + final byte[] bytes = new byte[length]; + input.readBytes(bytes, 0, length); + + components.put(name, new BytesRef(bytes)); + } + + SAICodecUtils.checkFooter(input); + } + + return new MetadataSource(version, components); + } + + public IndexInput get(String name) + { + BytesRef bytes = components.get(name); + + if (bytes == null) + { + throw new IllegalArgumentException(String.format("Could not find component '%s'. Available properties are %s.", + name, components.keySet())); + } + + return new ByteArrayIndexInput(name, bytes.bytes); + } + + public Version getVersion() + { + return version; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/MetadataWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/MetadataWriter.java new file mode 100644 index 000000000000..76434fae1637 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/MetadataWriter.java @@ -0,0 +1,86 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.cassandra.index.sai.disk.io.RAMIndexOutput; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; + +@NotThreadSafe +public class MetadataWriter implements Closeable +{ + private final IndexOutput output; + private final Map map = new HashMap<>(); + + public MetadataWriter(IndexOutput output) + { + this.output = output; + } + + public Builder builder(String name) + { + return new Builder(name); + } + + public class Builder extends RAMIndexOutput implements Closeable + { + private Builder(String name) + { + super(name); + } + + @Override + public void close() + { + map.put(getName(), new BytesRef(out.getBytes(), 0, out.getPosition())); + } + } + + private void finish() throws IOException + { + SAICodecUtils.writeHeader(output); + output.writeInt(map.size()); + for (Map.Entry entry : map.entrySet()) + { + output.writeString(entry.getKey()); + output.writeInt(entry.getValue().length); + output.writeBytes(entry.getValue().bytes, entry.getValue().offset, entry.getValue().length); + } + SAICodecUtils.writeFooter(output); + } + + @Override + public void close() throws IOException + { + try + { + finish(); + } + finally + { + output.close(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/MonotonicBlockPackedReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/MonotonicBlockPackedReader.java new file mode 100644 index 000000000000..cbbe18e9c692 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/MonotonicBlockPackedReader.java @@ -0,0 +1,132 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.io.IndexInputReader; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedLongValues; + +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.checkBlockSize; +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.numBlocks; + +/** + * Provides non-blocking, random access to a stream written with {@link MonotonicBlockPackedWriter}. + */ +public class MonotonicBlockPackedReader implements LongArray.Factory +{ + private final IndexComponents components; + private final FileHandle file; + private final int blockShift, blockMask; + private final long valueCount; + private final byte[] blockBitsPerValue; + private final PackedLongValues blockOffsets; + private final PackedLongValues minValues; + private final float[] averages; + + public MonotonicBlockPackedReader(FileHandle file, Component component, IndexComponents components, MetadataSource source) throws IOException + { + this(file, components, new NumericValuesMeta(source.get(component.name()))); + } + + @SuppressWarnings("resource") + public MonotonicBlockPackedReader(FileHandle file, IndexComponents components, NumericValuesMeta meta) throws IOException + { + this.components = components; + this.valueCount = meta.valueCount; + blockShift = checkBlockSize(meta.blockSize, AbstractBlockPackedWriter.MIN_BLOCK_SIZE, AbstractBlockPackedWriter.MAX_BLOCK_SIZE); + blockMask = meta.blockSize - 1; + int numBlocks = numBlocks(valueCount, meta.blockSize); + PackedLongValues.Builder minValuesBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT); + PackedLongValues.Builder blockOffsetsBuilder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT); + averages = new float[numBlocks]; + blockBitsPerValue = new byte[numBlocks]; + this.file = file; + + try (final RandomAccessReader reader = this.file.createReader()) + { + final IndexInputReader in = IndexInputReader.create(reader); + SAICodecUtils.validate(in); + + in.seek(meta.blockMetaOffset); + for (int i = 0; i < numBlocks; ++i) + { + minValuesBuilder.add(in.readZLong()); + averages[i] = Float.intBitsToFloat(in.readInt()); + final int bitsPerValue = in.readVInt(); + if (bitsPerValue > 64) + { + throw new CorruptIndexException(String.format("Block %d is corrupted. Bits per value should be no more than 64 and is %d.", i, bitsPerValue), in); + } + blockBitsPerValue[i] = (byte) bitsPerValue; + // when bitsPerValue is 0, block offset won't be used + blockOffsetsBuilder.add(bitsPerValue == 0 ? -1 : in.readVLong()); + } + } + + blockOffsets = blockOffsetsBuilder.build(); + minValues = minValuesBuilder.build(); + } + + @Override + @SuppressWarnings("resource") + public LongArray open() + { + final IndexInput indexInput = components.openInput(file); + return new AbstractBlockPackedReader(indexInput, blockBitsPerValue, blockShift, blockMask, 0, valueCount) + { + @Override + long delta(int block, int idx) + { + return expected(minValues.get(block), averages[block], idx); + } + + @Override + public void close() throws IOException + { + indexInput.close(); + } + + @Override + protected long blockOffsetAt(int block) + { + return blockOffsets.get(block); + } + + @Override + public long findTokenRowID(long targetValue) + { + throw new UnsupportedOperationException(); + } + }; + } + + public static long expected(long origin, float average, int index) + { + return origin + (long) (average * (long) index); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/MonotonicBlockPackedWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/MonotonicBlockPackedWriter.java new file mode 100644 index 000000000000..268daaffbabd --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/MonotonicBlockPackedWriter.java @@ -0,0 +1,86 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.packed.DirectWriter; + +/** + * A writer for large monotonically increasing sequences of positive longs. + * + * Modified copy of {@link org.apache.lucene.util.packed.MonotonicBlockPackedWriter} to use {@link DirectWriter} for + * optimised reads that doesn't require seeking through the whole file to open a thread-exclusive reader. + */ +public class MonotonicBlockPackedWriter extends AbstractBlockPackedWriter +{ + public MonotonicBlockPackedWriter(IndexOutput out, int blockSize) + { + super(out, blockSize); + } + + @Override + public void add(long l) throws IOException + { + assert l >= 0; + super.add(l); + } + + @Override + protected void flush() throws IOException + { + assert off > 0; + + final float avg = off == 1 ? 0f : (float) (values[off - 1] - values[0]) / (off - 1); + long min = values[0]; + // adjust min so that all deltas will be positive + for (int i = 1; i < off; ++i) + { + final long actual = values[i]; + final long expected = MonotonicBlockPackedReader.expected(min, avg, i); + if (expected > actual) + { + min -= (expected - actual); + } + } + + long maxDelta = 0; + for (int i = 0; i < off; ++i) + { + values[i] = values[i] - MonotonicBlockPackedReader.expected(min, avg, i); + maxDelta = Math.max(maxDelta, values[i]); + } + + blockMetaWriter.writeZLong(min); + blockMetaWriter.writeInt(Float.floatToIntBits(avg)); + if (maxDelta == 0) + { + blockMetaWriter.writeVInt(0); + } + else + { + final int bitsRequired = DirectWriter.bitsRequired(maxDelta); + blockMetaWriter.writeVInt(bitsRequired); + blockMetaWriter.writeVLong(out.getFilePointer()); + writeValues(off, bitsRequired); + } + + off = 0; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/NumericIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/NumericIndexWriter.java new file mode 100644 index 000000000000..51793c860e7f --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/NumericIndexWriter.java @@ -0,0 +1,195 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import com.google.common.base.MoreObjects; + +import org.apache.cassandra.index.sai.disk.IndexWriterConfig; +import org.apache.cassandra.index.sai.disk.MutableOneDimPointValues; +import org.apache.cassandra.index.sai.disk.SegmentMetadata; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.lucene.codecs.MutablePointValues; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.IOUtils; +import org.apache.lucene.util.packed.PackedInts; +import org.apache.lucene.util.packed.PackedLongValues; + +import static com.google.common.base.Preconditions.checkArgument; + + +/** + * Specialized writer for 1-dim point values, that builds them into a BKD tree with auxiliary posting lists on eligible + * tree levels. + * + * Given sorted input {@link MutablePointValues}, 1-dim case allows to optimise flush process, because we don't need to + * buffer all point values to sort them. + */ +public class NumericIndexWriter implements Closeable +{ + public static final int MAX_POINTS_IN_LEAF_NODE = BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE; + private final BKDWriter writer; + private final IndexComponents indexComponents; + private final int bytesPerDim; + private final boolean segmented; + + private final IndexWriterConfig config; + + /** + * @param maxSegmentRowId maximum possible segment row ID, used to create `maxDoc` for kd-tree + * @param numRows must be greater than number of added rowIds, only used for validation. + */ + public NumericIndexWriter(IndexComponents indexComponents, int bytesPerDim, long maxSegmentRowId, long numRows, IndexWriterConfig config, boolean segmented) throws IOException + { + this(indexComponents, MAX_POINTS_IN_LEAF_NODE, bytesPerDim, maxSegmentRowId, numRows, config, segmented); + } + + public NumericIndexWriter(IndexComponents indexComponents, int maxPointsInLeafNode, int bytesPerDim, long maxSegmentRowId, long numRows, IndexWriterConfig config, boolean segmented) throws IOException + { + checkArgument(maxSegmentRowId >= 0, + "[%s] maxRowId must be non-negative value, but got %s", + config.getIndexName(), maxSegmentRowId); + + checkArgument(numRows >= 0, + "[$s] numRows must be non-negative value, but got %s", + config.getIndexName(), numRows); + + this.indexComponents = indexComponents; + this.bytesPerDim = bytesPerDim; + this.config = config; + this.writer = new BKDWriter(maxSegmentRowId + 1, + 1, + bytesPerDim, + maxPointsInLeafNode, + BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP, + numRows, + true, null); + this.segmented = segmented; + } + + @Override + public void close() throws IOException + { + IOUtils.close(writer); + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("bytesPerDim", bytesPerDim) + .add("bufferedPoints", writer.getPointCount()) + .toString(); + } + + public static class LeafCallback implements BKDWriter.OneDimensionBKDWriterCallback + { + final List postings = new ArrayList<>(); + + public int numLeaves() + { + return postings.size(); + } + + @Override + public void writeLeafDocs(int leafNum, BKDWriter.RowIDAndIndex[] sortedByRowID, int offset, int count) + { + final PackedLongValues.Builder builder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT); + + for (int i = offset; i < count; ++i) + { + builder.add(sortedByRowID[i].rowID); + } + postings.add(builder.build()); + } + } + + /** + * Writes a k-d tree and posting lists from a {@link MutablePointValues}. + * + * @param values points to write + * + * @return metadata describing the location and size of this kd-tree in the overall SSTable kd-tree component file + */ + public SegmentMetadata.ComponentMetadataMap writeAll(MutableOneDimPointValues values) throws IOException + { + long bkdPosition; + final SegmentMetadata.ComponentMetadataMap components = new SegmentMetadata.ComponentMetadataMap(); + + final LeafCallback leafCallback = new LeafCallback(); + + try (IndexOutput bkdOutput = indexComponents.createOutput(indexComponents.kdTree, true, segmented)) + { + // The SSTable kd-tree component file is opened in append mode, so our offset is the current file pointer. + final long bkdOffset = bkdOutput.getFilePointer(); + + bkdPosition = writer.writeField(bkdOutput, values, leafCallback); + + // If the bkdPosition is less than 0 then we didn't write any values out + // and the index is empty + if (bkdPosition < 0) + return components; + + final long bkdLength = bkdOutput.getFilePointer() - bkdOffset; + + Map attributes = new LinkedHashMap<>(); + attributes.put("max_points_in_leaf_node", Integer.toString(writer.maxPointsInLeafNode)); + attributes.put("num_leaves", Integer.toString(leafCallback.numLeaves())); + attributes.put("num_points", Long.toString(writer.pointCount)); + attributes.put("bytes_per_dim", Long.toString(writer.bytesPerDim)); + attributes.put("num_dims", Long.toString(writer.numDims)); + + components.put(IndexComponents.NDIType.KD_TREE, bkdPosition, bkdOffset, bkdLength, attributes); + } + + try (TraversingBKDReader reader = new TraversingBKDReader(indexComponents, indexComponents.createFileHandle(indexComponents.kdTree, segmented), bkdPosition); + IndexOutput postingsOutput = indexComponents.createOutput(indexComponents.kdTreePostingLists, true, segmented)) + { + final long postingsOffset = postingsOutput.getFilePointer(); + + final OneDimBKDPostingsWriter postingsWriter = new OneDimBKDPostingsWriter(leafCallback.postings, config, indexComponents); + reader.traverse(postingsWriter); + + // The kd-tree postings writer already writes its own header & footer. + final long postingsPosition = postingsWriter.finish(postingsOutput); + + Map attributes = new LinkedHashMap<>(); + attributes.put("num_leaf_postings", Integer.toString(postingsWriter.numLeafPostings)); + attributes.put("num_non_leaf_postings", Integer.toString(postingsWriter.numNonLeafPostings)); + + long postingsLength = postingsOutput.getFilePointer() - postingsOffset; + components.put(IndexComponents.NDIType.KD_TREE_POSTING_LISTS, postingsPosition, postingsOffset, postingsLength, attributes); + } + + return components; + } + + /** + * @return number of points added + */ + public long getPointCount() + { + return writer.getPointCount(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/NumericValuesMeta.java b/src/java/org/apache/cassandra/index/sai/disk/v1/NumericValuesMeta.java new file mode 100644 index 000000000000..a27047955861 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/NumericValuesMeta.java @@ -0,0 +1,51 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; + +public class NumericValuesMeta +{ + final long valueCount; + final int blockSize; + final long blockMetaOffset; + + NumericValuesMeta(IndexInput input) throws IOException + { + valueCount = input.readLong(); + blockSize = input.readInt(); + blockMetaOffset = input.readVLong(); + } + + public NumericValuesMeta(long valueCount, int blockSize, long blockMetaOffset) + { + this.valueCount = valueCount; + this.blockSize = blockSize; + this.blockMetaOffset = blockMetaOffset; + } + + public void write(IndexOutput out) throws IOException + { + out.writeLong(valueCount); + out.writeInt(blockSize); + out.writeVLong(blockMetaOffset); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/NumericValuesWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/NumericValuesWriter.java new file mode 100644 index 000000000000..7da8888474ff --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/NumericValuesWriter.java @@ -0,0 +1,94 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.io.sstable.Component; +import org.apache.lucene.store.IndexOutput; + + +public class NumericValuesWriter implements Closeable +{ + public static final int MONOTONIC_BLOCK_SIZE = 16384; + public static final int BLOCK_SIZE = 128; + + private final IndexOutput output; + private final AbstractBlockPackedWriter writer; + private final MetadataWriter metadataWriter; + private final Component component; + private final int blockSize; + private long count = 0; + + public NumericValuesWriter(IndexComponents.IndexComponent component, + IndexOutput indexOutput, + MetadataWriter metadataWriter, + boolean monotonic) throws IOException + { + this(component, indexOutput, metadataWriter, monotonic, monotonic ? MONOTONIC_BLOCK_SIZE : BLOCK_SIZE); + } + + NumericValuesWriter(IndexComponents.IndexComponent component, + IndexComponents indexComponents, + MetadataWriter metadataWriter, + boolean monotonic, + int blockSize) throws IOException + { + this(component, indexComponents.createOutput(component), metadataWriter, monotonic, blockSize); + } + + private NumericValuesWriter(Component component, + IndexOutput indexOutput, + MetadataWriter metadataWriter, + boolean monotonic, int blockSize) throws IOException + { + SAICodecUtils.writeHeader(indexOutput); + this.writer = monotonic ? new MonotonicBlockPackedWriter(indexOutput, blockSize) + : new BlockPackedWriter(indexOutput, blockSize); + this.output = indexOutput; + this.component = component; + this.metadataWriter = metadataWriter; + this.blockSize = blockSize; + } + + @Override + public void close() throws IOException + { + try (IndexOutput o = metadataWriter.builder(component.name)) + { + final long fp = writer.finish(); + SAICodecUtils.writeFooter(output); + + NumericValuesMeta meta = new NumericValuesMeta(count, blockSize, fp); + meta.write(o); + } + finally + { + output.close(); + } + } + + public void add(long value) throws IOException + { + writer.add(value); + count++; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/OneDimBKDPostingsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/OneDimBKDPostingsWriter.java new file mode 100644 index 000000000000..57a94d739bf6 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/OneDimBKDPostingsWriter.java @@ -0,0 +1,186 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import com.google.common.base.Stopwatch; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.agrona.collections.IntArrayList; +import org.apache.cassandra.index.sai.disk.IndexWriterConfig; +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.packed.PackedLongValues; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +/** + * Writes auxiliary posting lists for bkd tree nodes. If a node has a posting list attached, it will contain every row + * id + * from all leaves reachable from that node. + * + * Writer is stateful, because it needs to collect data from bkd index data structure first to find set of eligible + * nodes and leaf nodes reachable from them. + * + * This is an optimised writer for 1-dim points, where we know that leaf blocks are written in value order (in this + * order we pass them to the {@link BKDWriter}). That allows us to skip reading the leaves, instead just order leaf + * blocks by their offset in the index file, and correlate them with buffered posting lists. We can't make this + * assumption for multi-dim case. + */ +public class OneDimBKDPostingsWriter implements TraversingBKDReader.IndexTreeTraversalCallback +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final List postings; + private final TreeMap leafOffsetToNodeID = new TreeMap<>(Long::compareTo); + private final Multimap nodeToChildLeaves = HashMultimap.create(); + + private final IndexWriterConfig config; + private final IndexComponents components; + int numNonLeafPostings = 0; + int numLeafPostings = 0; + + OneDimBKDPostingsWriter(List postings, IndexWriterConfig config, IndexComponents indexComponents) + { + this.postings = postings; + this.config = config; + this.components = indexComponents; + } + + @Override + public void onLeaf(int leafNodeID, long leafBlockFP, IntArrayList pathToRoot) + { + checkArgument(!pathToRoot.containsInt(leafNodeID)); + checkArgument(pathToRoot.isEmpty() || leafNodeID > pathToRoot.get(pathToRoot.size() - 1)); + + leafOffsetToNodeID.put(leafBlockFP, leafNodeID); + for (int i = 0; i < pathToRoot.size(); i++) + { + final int level = i + 1; + if (isLevelEligibleForPostingList(level)) + { + final int nodeID = pathToRoot.get(i); + nodeToChildLeaves.put(nodeID, leafNodeID); + } + } + } + + @SuppressWarnings("resource") + public long finish(IndexOutput out) throws IOException + { + checkState(postings.size() == leafOffsetToNodeID.size(), + "Expected equal number of postings lists (%s) and leaf offsets (%s).", + postings.size(), leafOffsetToNodeID.size()); + + final PostingsWriter postingsWriter = new PostingsWriter(out); + + final Iterator postingsIterator = postings.iterator(); + final Map leafToPostings = new HashMap<>(); + leafOffsetToNodeID.forEach((fp, nodeID) -> leafToPostings.put(nodeID, postingsIterator.next())); + + final long postingsRamBytesUsed = postings.stream() + .mapToLong(PackedLongValues::ramBytesUsed) + .sum(); + + final List internalNodeIDs = + nodeToChildLeaves.keySet() + .stream() + .filter(i -> nodeToChildLeaves.get(i).size() >= config.getBkdPostingsMinLeaves()) + .collect(Collectors.toList()); + + final Collection leafNodeIDs = leafOffsetToNodeID.values(); + + logger.debug(components.logMessage("Writing posting lists for {} internal and {} leaf kd-tree nodes. Leaf postings memory usage: {}."), + internalNodeIDs.size(), leafNodeIDs.size(), FBUtilities.prettyPrintMemory(postingsRamBytesUsed)); + + final long startFP = out.getFilePointer(); + final Stopwatch flushTime = Stopwatch.createStarted(); + final TreeMap nodeIDToPostingsFilePointer = new TreeMap<>(); + for (int nodeID : Iterables.concat(internalNodeIDs, leafNodeIDs)) + { + Collection leaves = nodeToChildLeaves.get(nodeID); + + if (leaves.size() == 0) + { + leaves = Collections.singletonList(nodeID); + numLeafPostings++; + } + else + { + numNonLeafPostings++; + } + + final PriorityQueue postingLists = new PriorityQueue<>(100, Comparator.comparingLong(PostingList.PeekablePostingList::peek)); + for (Integer leaf : leaves) + postingLists.add(new PackedLongsPostingList(leafToPostings.get(leaf)).peekable()); + + final PostingList mergedPostingList = MergePostingList.merge(postingLists); + final long postingFilePosition = postingsWriter.write(mergedPostingList); + // During compaction we could end up with an empty postings due to deletions. + // The writer will return a fp of -1 if no postings were written. + if (postingFilePosition >= 0) + nodeIDToPostingsFilePointer.put(nodeID, postingFilePosition); + } + flushTime.stop(); + logger.debug(components.logMessage("Flushed {} of posting lists for kd-tree nodes in {} ms."), + FBUtilities.prettyPrintMemory(out.getFilePointer() - startFP), + flushTime.elapsed(TimeUnit.MILLISECONDS)); + + + final long indexFilePointer = out.getFilePointer(); + writeMap(nodeIDToPostingsFilePointer, out); + postingsWriter.complete(); + return indexFilePointer; + } + + private boolean isLevelEligibleForPostingList(int level) + { + return level > 1 && level % config.getBkdPostingsSkip() == 0; + } + + private void writeMap(Map map, IndexOutput out) throws IOException + { + out.writeVInt(map.size()); + + for (Map.Entry e : map.entrySet()) + { + out.writeVInt(e.getKey()); + out.writeVLong(e.getValue()); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/OrdinalPostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/OrdinalPostingList.java new file mode 100644 index 000000000000..6e0fb035bf3b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/OrdinalPostingList.java @@ -0,0 +1,29 @@ +/* + * 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.index.sai.disk.v1; + +import org.apache.cassandra.index.sai.disk.PostingList; + +public interface OrdinalPostingList extends PostingList +{ + /** + * + * @return the ordinal of the posting that will be returned on the next call to {@link #nextPosting()} + */ + long getOrdinal(); +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PackedLongsPostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PackedLongsPostingList.java new file mode 100644 index 000000000000..61f9ae9cf450 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PackedLongsPostingList.java @@ -0,0 +1,64 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; + +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.lucene.util.packed.PackedLongValues; + +/** + * Adapter class for {@link PackedLongValues} to expose it as {@link PostingList}. + */ +public class PackedLongsPostingList implements PostingList +{ + private final PackedLongValues.Iterator iterator; + private final PackedLongValues values; + + PackedLongsPostingList(PackedLongValues values) + { + this.values = values; + iterator = values.iterator(); + } + + @Override + public long nextPosting() + { + if (iterator.hasNext()) + { + return iterator.next(); + } + else + { + return PostingList.END_OF_STREAM; + } + } + + @Override + public long size() + { + return values.size(); + } + + @Override + public long advance(long targetRowID) throws IOException + { + throw new UnsupportedOperationException(); + } +} + diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PostingsReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PostingsReader.java new file mode 100644 index 000000000000..f2df0941e4ac --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PostingsReader.java @@ -0,0 +1,397 @@ +/* + * 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.index.sai.disk.v1; + + +import java.io.IOException; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.cassandra.index.sai.metrics.QueryEventListener; +import org.apache.cassandra.index.sai.utils.LongArray; +import org.apache.cassandra.index.sai.utils.SeekingRandomAccessInput; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; + + +/** + * Reads, decompresses and decodes postings lists written by {@link PostingsWriter}. + * + * Holds exactly one postings block in memory at a time. Does binary search over skip table to find a postings block to + * load. + */ +@NotThreadSafe +public class PostingsReader implements OrdinalPostingList +{ + protected final IndexInput input; + private final int blockSize; + private final long numPostings; + private final LongArray blockOffsets; + private final LongArray blockMaxValues; + private final SeekingRandomAccessInput seekingInput; + private final QueryEventListener.PostingListEventListener listener; + + // TODO: Expose more things through the summary, now that it's an actual field? + private final BlocksSummary summary; + + private int postingsBlockIdx; + private int blockIdx; // position in block + private long totalPostingsRead; + private long actualSegmentRowId; + + private long currentPosition; + private DirectReaders.Reader currentFORValues; + private long postingsDecoded = 0; + + @VisibleForTesting + PostingsReader(IndexInput input, long summaryOffset, QueryEventListener.PostingListEventListener listener) throws IOException + { + this(input, new BlocksSummary(input, summaryOffset, () -> {}), listener); + } + + @VisibleForTesting + public PostingsReader(IndexInput input, BlocksSummary summary, QueryEventListener.PostingListEventListener listener) throws IOException + { + this.input = input; + this.seekingInput = new SeekingRandomAccessInput(input); + this.blockOffsets = summary.offsets; + this.blockSize = summary.blockSize; + this.numPostings = summary.numPostings; + this.blockMaxValues = summary.maxValues; + this.listener = listener; + + this.summary = summary; + + reBuffer(); + } + + @Override + public long getOrdinal() + { + return totalPostingsRead; + } + + interface InputCloser + { + void close() throws IOException; + } + + @VisibleForTesting + public static class BlocksSummary + { + final int blockSize; + final int numPostings; + final LongArray offsets; + final LongArray maxValues; + + private final InputCloser runOnClose; + + @VisibleForTesting + public BlocksSummary(IndexInput input, long offset) throws IOException + { + this(input, offset, input::close); + } + + BlocksSummary(IndexInput input, long offset, InputCloser runOnClose) throws IOException + { + this.runOnClose = runOnClose; + + input.seek(offset); + this.blockSize = input.readVInt(); + //TODO This should need to change because we can potentially end up with postings of more than Integer.MAX_VALUE? + this.numPostings = input.readVInt(); + + final SeekingRandomAccessInput randomAccessInput = new SeekingRandomAccessInput(input); + final int numBlocks = input.readVInt(); + final long maxBlockValuesLength = input.readVLong(); + final long maxBlockValuesOffset = input.getFilePointer() + maxBlockValuesLength; + + final byte offsetBitsPerValue = input.readByte(); + if (offsetBitsPerValue > 64) + { + throw new CorruptIndexException( + String.format("Postings list header is corrupted: Bits per value for block offsets must be no more than 64 and is %d.", offsetBitsPerValue), input); + } + this.offsets = new LongArrayReader(randomAccessInput, DirectReaders.getReaderForBitsPerValue(offsetBitsPerValue), input.getFilePointer(), numBlocks); + + input.seek(maxBlockValuesOffset); + final byte valuesBitsPerValue = input.readByte(); + if (valuesBitsPerValue > 64) + { + throw new CorruptIndexException( + String.format("Postings list header is corrupted: Bits per value for values samples must be no more than 64 and is %d.", valuesBitsPerValue), input); + } + this.maxValues = new LongArrayReader(randomAccessInput, DirectReaders.getReaderForBitsPerValue(valuesBitsPerValue), input.getFilePointer(), numBlocks); + } + + void close() throws IOException + { + runOnClose.close(); + } + + private static class LongArrayReader implements LongArray + { + private final RandomAccessInput input; + private final DirectReaders.Reader reader; + private final long offset; + private final int length; + + private LongArrayReader(RandomAccessInput input, DirectReaders.Reader reader, long offset, int length) + { + this.input = input; + this.reader = reader; + this.offset = offset; + this.length = length; + } + + @Override + public long findTokenRowID(long value) + { + throw new UnsupportedOperationException(); + } + + @Override + public long get(long idx) + { + return reader.get(input, offset, idx); + } + + @Override + public long length() + { + return length; + } + } + } + + @Override + public void close() throws IOException + { + listener.postingDecoded(postingsDecoded); + try + { + input.close(); + } + finally + { + summary.close(); + } + } + + @Override + public long size() + { + return numPostings; + } + + /** + * Advances to the first row ID beyond the current that is greater than or equal to the + * target, and returns that row ID. Exhausts the iterator and returns {@link #END_OF_STREAM} if + * the target is greater than the highest row ID. + * + * Does binary search over the skip table to find the next block to load into memory. + * + * Note: Callers must use the return value of this method before calling {@link #nextPosting()}, as calling + * that method will return the next posting, not the one to which we have just advanced. + * + * @param targetRowID target row ID to advance to + * + * @return first segment row ID which is >= the target row ID or {@link PostingList#END_OF_STREAM} if one does not exist + */ + @Override + public long advance(long targetRowID) throws IOException + { + listener.onAdvance(); + int block = binarySearchBlock(targetRowID); + + if (block < 0) + { + block = -block - 1; + } + + if (postingsBlockIdx == block + 1) + { + // we're in the same block, just iterate through + return slowAdvance(targetRowID); + } + assert block > 0; + // Even if there was an exact match, block might contain duplicates. + // We iterate to the target token from the beginning. + lastPosInBlock(block - 1); + return slowAdvance(targetRowID); + } + + private long slowAdvance(long targetRowID) throws IOException + { + while (totalPostingsRead < numPostings) + { + long segmentRowId = peekNext(); + + advanceOnePosition(segmentRowId); + + if (segmentRowId >= targetRowID) + { + return segmentRowId; + } + } + return END_OF_STREAM; + } + + private int binarySearchBlock(long targetRowID) + { + int low = postingsBlockIdx - 1; + int high = Math.toIntExact(blockMaxValues.length()) - 1; + + // in current block + if (low <= high && targetRowID <= blockMaxValues.get(low)) + return low; + + while (low <= high) + { + int mid = low + ((high - low) >> 1) ; + + long midVal = blockMaxValues.get(mid); + + if (midVal < targetRowID) + { + low = mid + 1; + } + else if (midVal > targetRowID) + { + high = mid - 1; + } + else + { + // target found, but we need to check for duplicates + if (mid > 0 && blockMaxValues.get(mid - 1L) == targetRowID) + { + // there are duplicates, pivot left + high = mid - 1; + } + else + { + // no duplicates + return mid; + } + } + } + return -(low + 1); // target not found + } + + private void lastPosInBlock(int block) + { + // blockMaxValues is integer only + actualSegmentRowId = blockMaxValues.get(block); + //upper bound, since we might've advanced to the last block, but upper bound is enough + totalPostingsRead += (blockSize - blockIdx) + (block - postingsBlockIdx + 1) * blockSize; + + postingsBlockIdx = block + 1; + blockIdx = blockSize; + } + + @Override + public long nextPosting() throws IOException + { + final long next = peekNext(); + if (next != END_OF_STREAM) + { + advanceOnePosition(next); + } + return next; + } + + @VisibleForTesting + int getBlockSize() + { + return blockSize; + } + + private long peekNext() throws IOException + { + if (totalPostingsRead >= numPostings) + { + return END_OF_STREAM; + } + if (blockIdx == blockSize) + { + reBuffer(); + } + + return actualSegmentRowId + nextRowID(); + } + + private int nextRowID() + { + // currentFORValues is null when the all the values in the block are the same + if (currentFORValues == null) + { + return 0; + } + else + { + final long id = currentFORValues.get(seekingInput, currentPosition, blockIdx); + postingsDecoded++; + return Math.toIntExact(id); + } + } + + private void advanceOnePosition(long nextRowID) + { + actualSegmentRowId = nextRowID; + totalPostingsRead++; + blockIdx++; + } + + private void reBuffer() throws IOException + { + final long pointer = blockOffsets.get(postingsBlockIdx); + + input.seek(pointer); + + final long left = numPostings - totalPostingsRead; + assert left > 0; + + readFoRBlock(input); + + postingsBlockIdx++; + blockIdx = 0; + } + + private void readFoRBlock(IndexInput in) throws IOException + { + final byte bitsPerValue = in.readByte(); + + currentPosition = in.getFilePointer(); + + if (bitsPerValue == 0) + { + // currentFORValues is null when the all the values in the block are the same + currentFORValues = null; + return; + } + else if (bitsPerValue > 64) + { + throw new CorruptIndexException( + String.format("Postings list #%s block is corrupted. Bits per value should be no more than 64 and is %d.", postingsBlockIdx, bitsPerValue), input); + } + currentFORValues = DirectReaders.getReaderForBitsPerValue(bitsPerValue); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PostingsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PostingsWriter.java new file mode 100644 index 000000000000..b86003dbc815 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PostingsWriter.java @@ -0,0 +1,313 @@ +/* + * 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.index.sai.disk.v1; + + +import java.io.Closeable; +import java.io.IOException; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; + +import org.agrona.collections.IntArrayList; +import org.agrona.collections.LongArrayList; +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.io.RAMIndexOutput; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.packed.DirectWriter; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.Math.max; +import static org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.BLOCK_SIZE; + +/** + * Encodes, compresses and writes postings lists to disk. + * + * All row IDs in the posting list are delta encoded, then deltas are divided into blocks for compression. + *

    + * In packed blocks, longs are encoded with the same bit width (FoR compression). The block size (i.e. number of + * longs inside block) is fixed (currently 128). Additionally blocks that are all the same value are encoded in an + * optimized way. + *

    + *

    + * In VLong blocks, longs are compressed with {@link DataOutput#writeVLong}. The block size is variable. + *

    + * + *

    + * Packed blocks are favoured, meaning when the postings are long enough, {@link PostingsWriter} will try + * to encode most data as a packed block. Take a term with 259 row IDs as an example, the first 256 IDs are encoded + * as two packed blocks, while the remaining 3 are encoded as one VLong block. + *

    + *

    + * Each posting list ends with a meta section and a skip table, that are written right after all postings blocks. Skip + * interval is the same as block size, and each skip entry points to the end of each block. Skip table consist of + * block offsets and last values of each block, compressed as two FoR blocks. + *

    + * + * Visual representation of the disk format: + *
    + *
    + * +========+========================+=====+==============+===============+============+=====+========================+========+
    + * | HEADER | POSTINGS LIST (TERM 1)                                                   | ... | POSTINGS LIST (TERM N) | FOOTER |
    + * +========+========================+=====+==============+===============+============+=====+========================+========+
    + *          | FOR BLOCK (1)          | ... | FOR BLOCK (N)| BLOCK SUMMARY              |
    + *          +------------------------+-----+--------------+---------------+------------+
    + *                                                        | BLOCK SIZE    |            |
    + *                                                        | LIST SIZE     | SKIP TABLE |
    + *                                                        +---------------+------------+
    + *                                                                        | BLOCKS POS.|
    + *                                                                        | MAX VALUES |
    + *                                                                        +------------+
    + *
    + *  
    + */ +@NotThreadSafe +//TODO Review this for DSP-19608 +public class PostingsWriter implements Closeable +{ + private final static String POSTINGS_MUST_BE_SORTED_ERROR_MSG = "Postings must be sorted ascending, got [%s] after [%s]"; + + private final IndexOutput dataOutput; + private final int blockSize; + private final long[] deltaBuffer; + private final LongArrayList blockOffsets = new LongArrayList(); + private final LongArrayList blockMaxIDs = new LongArrayList(); + private final RAMIndexOutput inMemoryOutput = new RAMIndexOutput("blockOffsets"); + + private final long startOffset; + + private int bufferUpto; + private long lastSegmentRowId; + private long maxDelta; + private long totalPostings; + + @VisibleForTesting + public PostingsWriter(IndexComponents components, boolean segmented) throws IOException + { + this(components, BLOCK_SIZE, segmented); + } + + PostingsWriter(IndexOutput dataOutput) throws IOException + { + this(dataOutput, BLOCK_SIZE); + } + + PostingsWriter(IndexComponents components, int blockSize, boolean segmented) throws IOException + { + this(components.createOutput(components.postingLists, true, segmented), blockSize); + } + + private PostingsWriter(IndexOutput dataOutput, int blockSize) throws IOException + { + this.blockSize = blockSize; + this.dataOutput = dataOutput; + startOffset = dataOutput.getFilePointer(); + deltaBuffer = new long[blockSize]; + SAICodecUtils.writeHeader(dataOutput); + } + + /** + * @return current file pointer + */ + public long getFilePointer() + { + return dataOutput.getFilePointer(); + } + + /** + * @return file pointer where index structure begins + */ + public long getStartOffset() + { + return startOffset; + } + + /** + * write footer to the postings + */ + public void complete() throws IOException + { + SAICodecUtils.writeFooter(dataOutput); + } + + @Override + public void close() throws IOException + { + dataOutput.close(); + } + + /** + * Encodes, compresses and flushes given posting list to disk. + * + * @param postings posting list to write to disk + * + * @return file offset to the summary block of this posting list + */ + public long write(PostingList postings) throws IOException + { + checkArgument(postings != null, "Expected non-null posting list."); + checkArgument(postings.size() > 0, "Expected non-empty posting list."); + + resetBlockCounters(); + blockOffsets.clear(); + blockMaxIDs.clear(); + + long segmentRowId; + // When postings list are merged, we don't know exact size, just an upper bound. + // We need to count how many postings we added to the block ourselves. + int size = 0; + while ((segmentRowId = postings.nextPosting()) != PostingList.END_OF_STREAM) + { + writePosting(segmentRowId); + size++; + totalPostings++; + } + if (size == 0) + return -1; + + finish(); + + final long summaryOffset = dataOutput.getFilePointer(); + writeSummary(size); + return summaryOffset; + } + + public long getTotalPostings() + { + return totalPostings; + } + + private void writePosting(long segmentRowId) throws IOException + { + if (!(segmentRowId >= lastSegmentRowId || lastSegmentRowId == 0)) + throw new IllegalArgumentException(String.format(POSTINGS_MUST_BE_SORTED_ERROR_MSG, segmentRowId, lastSegmentRowId)); + + final long delta = segmentRowId - lastSegmentRowId; + maxDelta = max(maxDelta, delta); + deltaBuffer[bufferUpto++] = delta; + + if (bufferUpto == blockSize) + { + addBlockToSkipTable(segmentRowId); + writePostingsBlock(maxDelta, bufferUpto); + resetBlockCounters(); + } + lastSegmentRowId = segmentRowId; + } + + private void finish() throws IOException + { + if (bufferUpto > 0) + { + addBlockToSkipTable(lastSegmentRowId); + + writePostingsBlock(maxDelta, bufferUpto); + } + } + + private void resetBlockCounters() + { + bufferUpto = 0; + lastSegmentRowId = 0; + maxDelta = 0; + } + + private void addBlockToSkipTable(long maxSegmentRowID) + { + blockOffsets.add(dataOutput.getFilePointer()); + blockMaxIDs.add(maxSegmentRowID); + } + + private void writeSummary(int exactSize) throws IOException + { + dataOutput.writeVInt(blockSize); + dataOutput.writeVInt(exactSize); + writeSkipTable(); + } + + private void writeSkipTable() throws IOException + { + assert blockOffsets.size() == blockMaxIDs.size(); + dataOutput.writeVInt(blockOffsets.size()); + + // compressing offsets in memory first, to know the exact length (with padding) + inMemoryOutput.reset(); + + writeSortedFoRBlock(blockOffsets, inMemoryOutput); + dataOutput.writeVLong(inMemoryOutput.getFilePointer()); + inMemoryOutput.writeTo(dataOutput); + writeSortedFoRBlock(blockMaxIDs, dataOutput); + } + + private void writePostingsBlock(long maxValue, int blockSize) throws IOException + { + final int bitsPerValue = maxValue == 0 ? 0 : DirectWriter.unsignedBitsRequired(maxValue); + + assert bitsPerValue < Byte.MAX_VALUE; + + dataOutput.writeByte((byte) bitsPerValue); + if (bitsPerValue > 0) + { + final DirectWriter writer = DirectWriter.getInstance(dataOutput, blockSize, bitsPerValue); + for (int i = 0; i < blockSize; ++i) + { + writer.add(deltaBuffer[i]); + } + writer.finish(); + } + } + + private void writeSortedFoRBlock(LongArrayList values, IndexOutput output) throws IOException + { + final long maxValue = values.getLong(values.size() - 1); + + assert values.size() > 0; + final int bitsPerValue = maxValue == 0 ? 0 : DirectWriter.unsignedBitsRequired(maxValue); + output.writeByte((byte) bitsPerValue); + if (bitsPerValue > 0) + { + final DirectWriter writer = DirectWriter.getInstance(output, values.size(), bitsPerValue); + for (int i = 0; i < values.size(); ++i) + { + writer.add(values.getLong(i)); + } + writer.finish(); + } + } + + private void writeSortedFoRBlock(IntArrayList values, IndexOutput output) throws IOException + { + final int maxValue = values.getInt(values.size() - 1); + + assert values.size() > 0; + final int bitsPerValue = maxValue == 0 ? 0 : DirectWriter.unsignedBitsRequired(maxValue); + output.writeByte((byte) bitsPerValue); + if (bitsPerValue > 0) + { + final DirectWriter writer = DirectWriter.getInstance(output, values.size(), bitsPerValue); + for (int i = 0; i < values.size(); ++i) + { + writer.add(values.getInt(i)); + } + writer.finish(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/TermsReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/TermsReader.java new file mode 100644 index 000000000000..70e0abe5052a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/TermsReader.java @@ -0,0 +1,306 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.concurrent.TimeUnit; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.index.sai.QueryContext; +import org.apache.cassandra.index.sai.disk.PostingList; +import org.apache.cassandra.index.sai.disk.TermsIterator; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.metrics.QueryEventListener; +import org.apache.cassandra.index.sai.utils.AbortedOperationException; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; +import org.apache.lucene.store.IndexInput; + +import static org.apache.cassandra.index.sai.utils.SAICodecUtils.validate; + +/** + * Synchronous reader of terms dictionary and postings lists to produce a {@link PostingList} with matching row ids. + * + * {@link #exactMatch(ByteComparable, QueryEventListener.TrieIndexEventListener, QueryContext)} does: + *
      + *
    • {@link TermQuery#lookupTermDictionary(ByteComparable)}: does term dictionary lookup to find the posting list file + * position
    • + *
    • {@link TermQuery#getPostingReader(long)}: reads posting list block summary and initializes posting read which + * reads the first block of the posting list into memory
    • + *
    + */ +public class TermsReader implements Closeable +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final IndexComponents indexComponents; + private final FileHandle termDictionaryFile; + private final FileHandle postingsFile; + private final long termDictionaryRoot; + + public TermsReader(IndexComponents components, FileHandle termsData, FileHandle postingLists, + long root, long termsFooterPointer) throws IOException + { + this.indexComponents = components; + termDictionaryFile = termsData; + postingsFile = postingLists; + termDictionaryRoot = root; + + try (final IndexInput indexInput = indexComponents.openInput(termDictionaryFile)) + { + // if the pointer is -1 then this is a previous version of the index + // use the old way to validate the footer + // the footer pointer is used due to encrypted indexes padding extra bytes + if (termsFooterPointer == -1) + { + validate(indexInput); + } + else + { + validate(indexInput, termsFooterPointer); + } + } + + try (final IndexInput indexInput = indexComponents.openInput(postingsFile)) + { + validate(indexInput); + } + } + + public static int openPerIndexFiles() + { + // terms and postings + return 2; + } + + @Override + public void close() + { + try + { + termDictionaryFile.close(); + } + finally + { + postingsFile.close(); + } + } + + public TermsIterator allTerms(long segmentOffset, QueryEventListener.TrieIndexEventListener listener) + { + // blocking, since we use it only for segment merging for now + return new TermsScanner(segmentOffset, listener); + } + + public PostingList exactMatch(ByteComparable term, QueryEventListener.TrieIndexEventListener perQueryEventListener, QueryContext context) + { + perQueryEventListener.onSegmentHit(); + return new TermQuery(term, perQueryEventListener, context).execute(); + } + + @VisibleForTesting + public class TermQuery + { + private final IndexInput postingsInput; + private final IndexInput postingsSummaryInput; + private final QueryEventListener.TrieIndexEventListener listener; + private final long lookupStartTime; + private final QueryContext context; + + private ByteComparable term; + + TermQuery(ByteComparable term, QueryEventListener.TrieIndexEventListener listener, QueryContext context) + { + this.listener = listener; + postingsInput = indexComponents.openInput(postingsFile); + postingsSummaryInput = indexComponents.openInput(postingsFile); + this.term = term; + lookupStartTime = System.nanoTime(); + this.context = context; + } + + public PostingList execute() + { + try + { + long postingOffset = lookupTermDictionary(term); + if (postingOffset == PostingList.OFFSET_NOT_FOUND) + { + FileUtils.closeQuietly(postingsInput); + FileUtils.closeQuietly(postingsSummaryInput); + return null; + } + + context.checkpoint(); + + // when posting is found, resources will be closed when posting reader is closed. + return getPostingReader(postingOffset); + } + catch (Throwable e) + { + //TODO Is there an equivalent of AOE in OS? + if (!(e instanceof AbortedOperationException)) + logger.error(indexComponents.logMessage("Failed to execute term query"), e); + + closeOnException(); + throw Throwables.cleaned(e); + } + } + + private void closeOnException() + { + FileUtils.closeQuietly(postingsInput); + FileUtils.closeQuietly(postingsSummaryInput); + } + + public long lookupTermDictionary(ByteComparable term) + { + try (TrieTermsDictionaryReader reader = new TrieTermsDictionaryReader(termDictionaryFile.instantiateRebufferer(), termDictionaryRoot)) + { + final long offset = reader.exactMatch(term); + + listener.onTraversalComplete(System.nanoTime() - lookupStartTime, TimeUnit.NANOSECONDS); + + if (offset == TrieTermsDictionaryReader.NOT_FOUND) + return PostingList.OFFSET_NOT_FOUND; + + return offset; + } + } + + public PostingsReader getPostingReader(long offset) throws IOException + { + PostingsReader.BlocksSummary header = new PostingsReader.BlocksSummary(postingsSummaryInput, offset); + + return new PostingsReader(postingsInput, header, listener.postingListEventListener()); + } + } + + // currently only used for testing + private class TermsScanner implements TermsIterator + { + private final long segmentOffset; + private final QueryEventListener.TrieIndexEventListener listener; + private final TrieTermsDictionaryReader termsDictionaryReader; + private final Iterator> iterator; + private final ByteBuffer minTerm, maxTerm; + private Pair entry; + + private TermsScanner(long segmentOffset, QueryEventListener.TrieIndexEventListener listener) + { + this.termsDictionaryReader = new TrieTermsDictionaryReader(termDictionaryFile.instantiateRebufferer(), termDictionaryRoot); + + this.minTerm = ByteBuffer.wrap(ByteSourceInverse.readBytes(termsDictionaryReader.getMinTerm().asComparableBytes(ByteComparable.Version.OSS41))); + this.maxTerm = ByteBuffer.wrap(ByteSourceInverse.readBytes(termsDictionaryReader.getMaxTerm().asComparableBytes(ByteComparable.Version.OSS41))); + this.iterator = termsDictionaryReader.iterator(); + this.listener = listener; + this.segmentOffset = segmentOffset; + } + + @Override + @SuppressWarnings("resource") + public PostingList postings() throws IOException + { + assert entry != null; + final IndexInput input = indexComponents.openInput(postingsFile); + return new OffsetPostingList(segmentOffset, new PostingsReader(input, new PostingsReader.BlocksSummary(input, entry.right), listener.postingListEventListener())); + } + + @Override + public void close() + { + termsDictionaryReader.close(); + } + + @Override + public ByteBuffer getMinTerm() + { + return minTerm; + } + + @Override + public ByteBuffer getMaxTerm() + { + return maxTerm; + } + + @Override + public ByteComparable next() + { + if (iterator.hasNext()) + { + entry = iterator.next(); + return entry.left; + } + return null; + } + + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + } + + private class OffsetPostingList implements PostingList + { + private final long offset; + private final PostingList wrapped; + + OffsetPostingList(long offset, PostingList postingList) + { + this.offset = offset; + this.wrapped = postingList; + } + + @Override + public long nextPosting() throws IOException + { + long next = wrapped.nextPosting(); + if (next == PostingList.END_OF_STREAM) + return next; + return next + offset; + } + + @Override + public long size() + { + return wrapped.size(); + } + + @Override + public long advance(long targetRowID) throws IOException + { + long next = wrapped.advance(targetRowID); + if (next == PostingList.END_OF_STREAM) + return next; + return next + offset; + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/TraversingBKDReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/TraversingBKDReader.java new file mode 100644 index 000000000000..a92b2808b050 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/TraversingBKDReader.java @@ -0,0 +1,468 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.Closeable; + +import org.agrona.collections.IntArrayList; +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.io.IndexInputReader; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.utils.ObjectSizes; +import org.apache.cassandra.utils.Throwables; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.ByteArrayDataInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.FutureArrays; +import org.apache.lucene.util.MathUtil; + +/** + * Base reader for a block KD-tree previously written with {@link BKDWriter}. + * + * Holds index tree on heap and enables it's traversal via {@link #traverse(IndexTreeTraversalCallback)}. + */ +public class TraversingBKDReader implements Closeable +{ + final IndexComponents indexComponents; + final FileHandle indexFile; + final int bytesPerDim; + final int numLeaves; + final byte[] minPackedValue; + final byte[] maxPackedValue; + // Packed array of byte[] holding all split values in the full binary tree: + final byte[] packedIndex; + final long pointCount; + final int leafNodeOffset; + final int numDims; + final int maxPointsInLeafNode; + final int packedBytesLength; + + @SuppressWarnings("resource") + TraversingBKDReader(IndexComponents indexComponents, FileHandle indexFile, long root) + { + this.indexComponents = indexComponents; + this.indexFile = indexFile; + + try (final RandomAccessReader reader = indexFile.createReader()) + { + final IndexInputReader in = IndexInputReader.create(reader); + SAICodecUtils.validate(in); + in.seek(root); + + numDims = in.readVInt(); + maxPointsInLeafNode = in.readVInt(); + bytesPerDim = in.readVInt(); + packedBytesLength = numDims * bytesPerDim; + + // Read index: + numLeaves = in.readVInt(); + assert numLeaves > 0; + leafNodeOffset = numLeaves; + + minPackedValue = new byte[packedBytesLength]; + maxPackedValue = new byte[packedBytesLength]; + +// if (indexComponents.getEncryptionCompressor() != null) +// { +// IndexInput cryptoInput = CryptoUtils.uncompress(in, indexComponents.getEncryptionCompressor()); +// cryptoInput.readBytes(minPackedValue, 0, packedBytesLength); +// cryptoInput.readBytes(maxPackedValue, 0, packedBytesLength); +// } +// else +// { + in.readBytes(minPackedValue, 0, packedBytesLength); + in.readBytes(maxPackedValue, 0, packedBytesLength); +// } + + for (int dim = 0; dim < numDims; dim++) + { + if (FutureArrays.compareUnsigned(minPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim, maxPackedValue, dim * bytesPerDim, dim * bytesPerDim + bytesPerDim) > 0) + { + String message = String.format("Min packed value %s is > max packed value %s for dimension %d.", + new BytesRef(minPackedValue), new BytesRef(maxPackedValue), dim); + throw new CorruptIndexException(message, in); + } + } + + pointCount = in.readVLong(); + + // docCount, unused + in.readVInt(); + +// ICompressor compressor = indexComponents.getEncryptionCompressor(); +// if (compressor != null) +// { +// // TODO: there's extra byte[] allocation here +// IndexInput input = CryptoUtils.uncompress(in, compressor); +// +// packedIndex = new byte[(int)input.length()]; +// input.readBytes(packedIndex, 0, (int)input.length()); +// } +// else +// { + int numBytes = in.readVInt(); + packedIndex = new byte[numBytes]; + in.readBytes(packedIndex, 0, numBytes); +// } + } + catch (Throwable t) + { + FileUtils.closeQuietly(indexFile); + throw Throwables.unchecked(t); + } + } + + public long getMinLeafBlockFP() + { + if (packedIndex != null) + { + return new ByteArrayDataInput(packedIndex).readVLong(); + } + else + { + throw new IllegalStateException(); + } + } + + public long memoryUsage() + { + return ObjectSizes.sizeOfArray(packedIndex) + + ObjectSizes.sizeOfArray(minPackedValue) + + ObjectSizes.sizeOfArray(maxPackedValue); + } + + @Override + public void close() + { + indexFile.close(); + } + + interface IndexTreeTraversalCallback + { + void onLeaf(int leafNodeID, long leafBlockFP, IntArrayList pathToRoot); + } + + /** + * Copy of BKDReader.IndexTree + */ + abstract class IndexTree implements Cloneable + { + protected int nodeID; + // level is 1-based so that we can do level-1 w/o checking each time: + protected int level; + protected int splitDim; + protected final byte[][] splitPackedValueStack; + + protected IndexTree() + { + int treeDepth = getTreeDepth(); + splitPackedValueStack = new byte[treeDepth + 1][]; + nodeID = 1; + level = 1; + splitPackedValueStack[level] = new byte[packedBytesLength]; + } + + public void pushLeft() + { + nodeID *= 2; + level++; + if (splitPackedValueStack[level] == null) + { + splitPackedValueStack[level] = new byte[packedBytesLength]; + } + } + + /** Clone, but you are not allowed to pop up past the point where the clone happened. */ + public abstract IndexTree clone(); + + public void pushRight() + { + nodeID = nodeID * 2 + 1; + level++; + if (splitPackedValueStack[level] == null) + { + splitPackedValueStack[level] = new byte[packedBytesLength]; + } + } + + public void pop() + { + nodeID /= 2; + level--; + splitDim = -1; + //System.out.println(" pop nodeID=" + nodeID); + } + + public boolean isLeafNode() + { + return nodeID >= leafNodeOffset; + } + + public boolean nodeExists() + { + return nodeID - leafNodeOffset < leafNodeOffset; + } + + public int getNodeID() + { + return nodeID; + } + + public byte[] getSplitPackedValue() + { + assert !isLeafNode(); + assert splitPackedValueStack[level] != null : "level=" + level; + return splitPackedValueStack[level]; + } + + /** Only valid after pushLeft or pushRight, not pop! */ + public int getSplitDim() + { + assert !isLeafNode(); + return splitDim; + } + + /** Only valid after pushLeft or pushRight, not pop! */ + public abstract BytesRef getSplitDimValue(); + + /** Only valid after pushLeft or pushRight, not pop! */ + public abstract long getLeafBlockFP(); + } + + + /** + * Copy of BKDReader.PackedIndexTree + */ + final class PackedIndexTree extends IndexTree + { + // used to read the packed byte[] + private final ByteArrayDataInput in; + // holds the minimum (left most) leaf block file pointer for each level we've recursed to: + private final long[] leafBlockFPStack; + // holds the address, in the packed byte[] index, of the left-node of each level: + private final int[] leftNodePositions; + // holds the address, in the packed byte[] index, of the right-node of each level: + private final int[] rightNodePositions; + // holds the splitDim for each level: + private final int[] splitDims; + // true if the per-dim delta we read for the node at this level is a negative offset vs. the last split on this dim; this is a packed + // 2D array, i.e. to access array[level][dim] you read from negativeDeltas[level*numDims+dim]. this will be true if the last time we + // split on this dimension, we next pushed to the left sub-tree: + private final boolean[] negativeDeltas; + // holds the packed per-level split values; the run method uses this to save the cell min/max as it recurses: + private final byte[][] splitValuesStack; + // scratch value to return from getPackedValue: + private final BytesRef scratch; + + PackedIndexTree() + { + int treeDepth = getTreeDepth(); + leafBlockFPStack = new long[treeDepth + 1]; + leftNodePositions = new int[treeDepth + 1]; + rightNodePositions = new int[treeDepth + 1]; + splitValuesStack = new byte[treeDepth + 1][]; + splitDims = new int[treeDepth + 1]; + negativeDeltas = new boolean[numDims * (treeDepth + 1)]; + + in = new ByteArrayDataInput(packedIndex); + splitValuesStack[0] = new byte[packedBytesLength]; + readNodeData(false); + scratch = new BytesRef(); + scratch.length = bytesPerDim; + } + + @Override + public PackedIndexTree clone() + { + PackedIndexTree index = new PackedIndexTree(); + index.nodeID = nodeID; + index.level = level; + index.splitDim = splitDim; + index.leafBlockFPStack[level] = leafBlockFPStack[level]; + index.leftNodePositions[level] = leftNodePositions[level]; + index.rightNodePositions[level] = rightNodePositions[level]; + index.splitValuesStack[index.level] = splitValuesStack[index.level].clone(); + System.arraycopy(negativeDeltas, level * numDims, index.negativeDeltas, level * numDims, numDims); + index.splitDims[level] = splitDims[level]; + return index; + } + + @Override + public void pushLeft() + { + int nodePosition = leftNodePositions[level]; + super.pushLeft(); + System.arraycopy(negativeDeltas, (level - 1) * numDims, negativeDeltas, level * numDims, numDims); + assert splitDim != -1; + negativeDeltas[level * numDims + splitDim] = true; + in.setPosition(nodePosition); + readNodeData(true); + } + + @Override + public void pushRight() + { + int nodePosition = rightNodePositions[level]; + super.pushRight(); + System.arraycopy(negativeDeltas, (level - 1) * numDims, negativeDeltas, level * numDims, numDims); + assert splitDim != -1; + negativeDeltas[level * numDims + splitDim] = false; + in.setPosition(nodePosition); + readNodeData(false); + } + + @Override + public void pop() + { + super.pop(); + splitDim = splitDims[level]; + } + + @Override + public long getLeafBlockFP() + { + assert isLeafNode() : "nodeID=" + nodeID + " is not a leaf"; + return leafBlockFPStack[level]; + } + + @Override + public BytesRef getSplitDimValue() + { + assert !isLeafNode(); + scratch.bytes = splitValuesStack[level]; + scratch.offset = splitDim * bytesPerDim; + return scratch; + } + + private void readNodeData(boolean isLeft) + { + + leafBlockFPStack[level] = leafBlockFPStack[level - 1]; + + // read leaf block FP delta + if (!isLeft) + { + leafBlockFPStack[level] += in.readVLong(); + } + + if (isLeafNode()) + { + splitDim = -1; + } + else + { + + // read split dim, prefix, firstDiffByteDelta encoded as int: + int code = in.readVInt(); + splitDim = code % numDims; + splitDims[level] = splitDim; + code /= numDims; + int prefix = code % (1 + bytesPerDim); + int suffix = bytesPerDim - prefix; + + if (splitValuesStack[level] == null) + { + splitValuesStack[level] = new byte[packedBytesLength]; + } + System.arraycopy(splitValuesStack[level - 1], 0, splitValuesStack[level], 0, packedBytesLength); + if (suffix > 0) + { + int firstDiffByteDelta = code / (1 + bytesPerDim); + if (negativeDeltas[level * numDims + splitDim]) + { + firstDiffByteDelta = -firstDiffByteDelta; + } + int oldByte = splitValuesStack[level][splitDim * bytesPerDim + prefix] & 0xFF; + splitValuesStack[level][splitDim * bytesPerDim + prefix] = (byte) (oldByte + firstDiffByteDelta); + in.readBytes(splitValuesStack[level], splitDim * bytesPerDim + prefix + 1, suffix - 1); + } + else + { + // our split value is == last split value in this dim, which can happen when there are many duplicate values + } + + int leftNumBytes; + if (nodeID * 2 < leafNodeOffset) + { + leftNumBytes = in.readVInt(); + } + else + { + leftNumBytes = 0; + } + + leftNodePositions[level] = in.getPosition(); + rightNodePositions[level] = leftNodePositions[level] + leftNumBytes; + } + } + } + + + void traverse(IndexTreeTraversalCallback callback) + { + traverse(callback, + new PackedIndexTree(), + new IntArrayList()); + } + + private void traverse(IndexTreeTraversalCallback callback, + IndexTree index, + IntArrayList pathToRoot) + { + if (index.isLeafNode()) + { + // In the unbalanced case it's possible the left most node only has one child: + if (index.nodeExists()) + { + callback.onLeaf(index.getNodeID(), index.getLeafBlockFP(), pathToRoot); + } + } + else + { + final int nodeID = index.getNodeID(); + final IntArrayList currentPath = new IntArrayList(); + currentPath.addAll(pathToRoot); + currentPath.add(nodeID); + + index.pushLeft(); + traverse(callback, index, currentPath); + index.pop(); + + index.pushRight(); + traverse(callback, index, currentPath); + index.pop(); + } + } + + /** + * Copy of BKDReader#getTreeDepth() + */ + private int getTreeDepth() + { + // First +1 because all the non-leave nodes makes another power + // of 2; e.g. to have a fully balanced tree with 4 leaves you + // need a depth=3 tree: + + // Second +1 because MathUtil.log computes floor of the logarithm; e.g. + // with 5 leaves you need a depth=4 tree: + return MathUtil.log(numLeaves, 2) + 2; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/TrieTermsDictionaryReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/TrieTermsDictionaryReader.java new file mode 100644 index 000000000000..c97dd44bc4c4 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/TrieTermsDictionaryReader.java @@ -0,0 +1,284 @@ +/* + * 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.index.sai.disk.v1; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.collect.AbstractIterator; + +import org.apache.cassandra.io.tries.SerializationNode; +import org.apache.cassandra.io.tries.TrieNode; +import org.apache.cassandra.io.tries.TrieSerializer; +import org.apache.cassandra.io.tries.Walker; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.io.util.Rebufferer; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.SizedInts; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.lucene.util.ArrayUtil; + +/** + * Page-aware random access reader for a trie terms dictionary written by {@link TrieTermsDictionaryWriter}. + */ +@NotThreadSafe +class TrieTermsDictionaryReader extends Walker +{ + static final long NOT_FOUND = -1; + + TrieTermsDictionaryReader(Rebufferer rebufferer, long root) + { + super(rebufferer, root); + } + + static final TrieSerializer trieSerializer = new TrieSerializer() + { + @Override + public int sizeofNode(SerializationNode node, long nodePosition) + { + return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload()); + } + + @Override + public void write(DataOutputPlus dest, SerializationNode node, long nodePosition) throws IOException + { + final TrieNode type = TrieNode.typeFor(node, nodePosition); + final Long payload = node.payload(); + if (payload != null) + { + final int payloadBits = SizedInts.nonZeroSize(payload); + type.serialize(dest, node, payloadBits, nodePosition); + SizedInts.write(dest, payload, payloadBits); + } + else + { + type.serialize(dest, node, 0, nodePosition); + } + } + + private int sizeof(Long payload) + { + if (payload != null) + { + return SizedInts.nonZeroSize(payload); + } + return 0; + } + }; + + long exactMatch(ByteComparable key) + { + int b = follow(key); + if (b != ByteSource.END_OF_STREAM) + { + return NOT_FOUND; + } + return getCurrentPayload(); + } + + Iterator> iterator() + { + return new AbstractIterator>() + { + final TransitionBytesCollector collector = new TransitionBytesCollector(); + IterationPosition stack = new IterationPosition(root, -1, null); + + @Override + protected Pair computeNext() + { + final long node = advanceNode(); + if (node == -1) + { + return endOfData(); + } + return Pair.create(collector.toByteComparable(), getCurrentPayload()); + } + + private long advanceNode() + { + long child; + int transitionByte; + + go(stack.node); + while (true) + { + int childIndex = stack.childIndex + 1; + transitionByte = transitionByte(childIndex); + + if (transitionByte > 256) + { + // ascend + stack = stack.prev; + collector.pop(); + if (stack == null) + { + // exhausted whole trie + return -1; + } + go(stack.node); + continue; + } + + child = transition(childIndex); + + if (child != -1) + { + assert child >= 0 : String.format("Expected value >= 0 but got %d - %s", child, this); + + // descend + go(child); + + stack.childIndex = childIndex; + stack = new IterationPosition(child, -1, stack); + collector.add(transitionByte); + + if (payloadFlags() != 0) + return child; + } + else + { + stack.childIndex = childIndex; + } + } + } + }; + } + + ByteComparable getMaxTerm() + { + final TransitionBytesCollector collector = new ImmutableTransitionBytesCollector(); + go(root); + while (true) + { + int lastIdx = transitionRange() - 1; + long lastChild = transition(lastIdx); + if (lastIdx < 0) + { + return collector.toByteComparable(); + } + collector.add(transitionByte(lastIdx)); + go(lastChild); + } + } + + ByteComparable getMinTerm() + { + final TransitionBytesCollector collector = new ImmutableTransitionBytesCollector(); + go(root); + while (true) + { + int payloadBits = payloadFlags(); + if (payloadBits > 0) + { + return collector.toByteComparable(); + } + collector.add(transitionByte(0)); + go(transition(0)); + } + } + + private long getCurrentPayload() + { + return getPayload(buf, payloadPosition(), payloadFlags()); + } + + private long getPayload(ByteBuffer contents, int payloadPos, int bytes) + { + if (bytes == 0) + { + return NOT_FOUND; + } + return SizedInts.read(contents, payloadPos, bytes); + } + + private static class ImmutableTransitionBytesCollector extends TransitionBytesCollector + { + @Override + ByteComparable toByteComparable() + { + assert pos > 0; + final int length = pos; + return v -> ByteSource.fixedLength(bytes, 0, length); + } + + @Override + void pop() + { + throw new UnsupportedOperationException(); + } + } + + private static class TransitionBytesCollector + { + protected byte[] bytes = new byte[32]; + protected int pos = 0; + + void add(int b) + { + if (pos == bytes.length) + { + bytes = ArrayUtil.grow(bytes, pos + 1); + } + bytes[pos++] = (byte) b; + } + + void pop() + { + assert pos >= 0; + pos--; + } + + ByteComparable toByteComparable() + { + assert pos > 0; + final byte[] value = new byte[pos]; + System.arraycopy(bytes, 0, value, 0, pos); + return v -> ByteSource.fixedLength(value, 0, value.length); + } + + @Override + public String toString() + { + return String.format("[Bytes %s, pos %d]", Arrays.toString(bytes), pos); + } + } + + private static class IterationPosition + { + final long node; + final IterationPosition prev; + int childIndex; + + IterationPosition(long node, int childIndex, IterationPosition prev) + { + this.node = node; + this.childIndex = childIndex; + this.prev = prev; + } + + @Override + public String toString() + { + return String.format("[Node %d, child %d]", node, childIndex); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/TrieTermsDictionaryWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/TrieTermsDictionaryWriter.java new file mode 100644 index 000000000000..712c669cc283 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/TrieTermsDictionaryWriter.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.index.sai.disk.v1; + +import java.io.Closeable; +import java.io.IOException; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.commons.lang3.mutable.MutableLong; + +import org.apache.cassandra.index.sai.disk.io.IndexComponents; +import org.apache.cassandra.index.sai.disk.io.IndexOutputWriter; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.io.tries.IncrementalDeepTrieWriterPageAware; +import org.apache.cassandra.io.tries.IncrementalTrieWriter; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +/** + * Writes terms dictionary to disk in a trie format (see {@link IncrementalTrieWriter}. + * + * Allows for variable-length keys. Trie values are 64-bit offsets to the posting file, pointing to the beginning of + * summary block for that postings list. + */ +@NotThreadSafe +public class TrieTermsDictionaryWriter implements Closeable +{ + private final IncrementalTrieWriter termsDictionaryWriter; + private final IndexOutputWriter termDictionaryOutput; + private final long startOffset; + + TrieTermsDictionaryWriter(IndexComponents indexComponents, boolean segmented) throws IOException + { + termDictionaryOutput = indexComponents.createOutput(indexComponents.termsData, true, segmented); + startOffset = termDictionaryOutput.getFilePointer(); + + SAICodecUtils.writeHeader(termDictionaryOutput); + // we pass the output as SequentialWriter, but we keep IndexOutputWriter around to write footer on flush + termsDictionaryWriter = new IncrementalDeepTrieWriterPageAware<>(TrieTermsDictionaryReader.trieSerializer, termDictionaryOutput.asSequentialWriter()); + } + + public void add(ByteComparable term, long postingListOffset) throws IOException + { + termsDictionaryWriter.add(term, postingListOffset); + } + + @Override + public void close() throws IOException + { + termsDictionaryWriter.close(); + termDictionaryOutput.close(); + } + + /** + * complete trie index and write footer + * + * @return the position in the file of the root node. + */ + public long complete(MutableLong footerPointer) throws IOException + { + long root = termsDictionaryWriter.complete(); + + footerPointer.setValue(termDictionaryOutput.getFilePointer()); + SAICodecUtils.writeFooter(termDictionaryOutput); + return root; + } + + /** + * @return current file pointer + */ + public long getFilePointer() + { + return termDictionaryOutput.getFilePointer(); + } + + /** + * @return file pointer where index structure begins + */ + public long getStartOffset() + { + return startOffset; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/memory/InMemoryToken.java b/src/java/org/apache/cassandra/index/sai/memory/InMemoryToken.java new file mode 100644 index 000000000000..3111ae577660 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/memory/InMemoryToken.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.index.sai.memory; + +import java.util.Collections; +import java.util.Iterator; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.collect.Iterators; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.Token; + +/** + * A single-use {@link Token} whose iterator of {@link DecoratedKey} is already materialized on-heap. + */ +@NotThreadSafe +public class InMemoryToken extends Token +{ + private final Iterator keys; + + public InMemoryToken(long token, Iterator keys) + { + super(token); + this.keys = keys == null ? Collections.emptyIterator() : keys; + } + + public InMemoryToken(long token, DecoratedKey key) + { + super(token); + this.keys = Iterators.singletonIterator(key); + } + + @Override + public Iterator keys() + { + return keys; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/memory/KeyRangeIterator.java b/src/java/org/apache/cassandra/index/sai/memory/KeyRangeIterator.java new file mode 100644 index 000000000000..972cdcb4e98b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/memory/KeyRangeIterator.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.index.sai.memory; + +import java.io.IOException; +import java.util.PriorityQueue; +import java.util.SortedSet; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.index.sai.utils.RangeIterator; + +public class KeyRangeIterator extends RangeIterator +{ + private final PriorityQueue keys; + private final boolean uniqueKeys; + private DecoratedKey lastKey; + + /** + * An in-memory {@link RangeIterator} that uses a {@link SortedSet} which has no duplication as its backing store. + */ + public KeyRangeIterator(SortedSet keys) + { + super((Long) keys.first().getToken().getTokenValue(), (Long) keys.last().getToken().getTokenValue(), keys.size()); + this.keys = new PriorityQueue<>(keys); + this.uniqueKeys = true; + } + + /** + * An in-memory {@link RangeIterator} that uses a {@link PriorityQueue} which may + * contain duplicated keys as its backing store. + */ + public KeyRangeIterator(Long min, Long max, PriorityQueue keys) + { + super(min, max, keys.size()); + this.keys = keys; + this.uniqueKeys = false; + } + + protected Token computeNext() + { + DecoratedKey key = computeNextKey(); + return key == null ? endOfData() : new InMemoryToken(key.getToken().getLongValue(), key); + } + + private DecoratedKey computeNextKey() + { + DecoratedKey next = null; + + while (!keys.isEmpty()) + { + DecoratedKey key = keys.poll(); + if (uniqueKeys) + return key; + + if (lastKey == null || lastKey.compareTo(key) != 0) + { + next = key; + lastKey = key; + break; + } + } + + return next; + } + + protected void performSkipTo(Long nextToken) + { + while (!keys.isEmpty()) + { + DecoratedKey key = keys.peek(); + if ((long) key.getToken().getTokenValue() >= nextToken) + break; + + // consume smaller key + keys.poll(); + } + } + + public void close() throws IOException + {} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/index/sai/memory/MemoryIndex.java b/src/java/org/apache/cassandra/index/sai/memory/MemoryIndex.java new file mode 100644 index 000000000000..398c4002191c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/memory/MemoryIndex.java @@ -0,0 +1,80 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.memory; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.PrimaryKeys; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +public abstract class MemoryIndex +{ + protected final ColumnContext columnContext; + + private ByteBuffer minTerm; + private ByteBuffer maxTerm; + + protected MemoryIndex(ColumnContext columnContext) + { + this.columnContext = columnContext; + } + + public abstract long add(DecoratedKey key, Clustering clustering, ByteBuffer value); + + public abstract RangeIterator search(Expression expression, AbstractBounds keyRange); + + public void setMinMaxTerm(ByteBuffer term) + { + assert term != null; + + minTerm = TypeUtil.min(term, minTerm, columnContext.getValidator()); + maxTerm = TypeUtil.max(term, maxTerm, columnContext.getValidator()); + } + + public ByteBuffer getMinTerm() + { + return minTerm; + } + + public ByteBuffer getMaxTerm() + { + return maxTerm; + } + + /** + * Iterate all Term->PrimaryKeys mappings in sorted order + */ + public abstract Iterator> iterator(); +} diff --git a/src/java/org/apache/cassandra/index/sai/memory/MemtableIndex.java b/src/java/org/apache/cassandra/index/sai/memory/MemtableIndex.java new file mode 100644 index 000000000000..320f3b13ced5 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/memory/MemtableIndex.java @@ -0,0 +1,105 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.memory; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.concurrent.atomic.LongAdder; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.PrimaryKeys; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; + +public class MemtableIndex +{ + private final MemoryIndex index; + private final AbstractType validator; + private final ClusteringComparator clusteringComparator; + private final LongAdder writeCount = new LongAdder(); + private final LongAdder estimatedMemoryUsed = new LongAdder(); + + public MemtableIndex(ColumnContext columnContext, Memtable mt) + { + this.index = new TrieMemoryIndex(columnContext); + this.validator = columnContext.getValidator(); + this.clusteringComparator = columnContext.clusteringComparator(); + } + + public long writeCount() + { + return writeCount.sum(); + } + + public long estimatedMemoryUsed() + { + return estimatedMemoryUsed.sum(); + } + + public boolean isEmpty() + { + return getMinTerm() == null; + } + + public ByteBuffer getMinTerm() + { + return index.getMinTerm(); + } + + public ByteBuffer getMaxTerm() + { + return index.getMaxTerm(); + } + + public long index(DecoratedKey key, Clustering clustering, ByteBuffer value) + { + if (value == null || value.remaining() == 0) + return 0; + + long ram = index.add(key, clustering, value); + writeCount.increment(); + estimatedMemoryUsed.add(ram); + return ram; + } + + public RangeIterator search(Expression expression, AbstractBounds keyRange) + { + return index.search(expression, keyRange); + } + + public Iterator> iterator() + { + return index.iterator(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/memory/RowMapping.java b/src/java/org/apache/cassandra/index/sai/memory/RowMapping.java new file mode 100644 index 000000000000..9241c7bd9625 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/memory/RowMapping.java @@ -0,0 +1,205 @@ +/* + * 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.index.sai.memory; + +import java.util.Collections; +import java.util.Iterator; + +import com.carrotsearch.hppc.IntArrayList; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.rows.RangeTombstoneMarker; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.tries.MemtableTrie; +import org.apache.cassandra.db.tries.Trie; +import org.apache.cassandra.index.sai.disk.SegmentBuilder; +import org.apache.cassandra.index.sai.utils.AbstractIterator; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.index.sai.utils.PrimaryKeys; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +/** + * In memory representation of {@link PrimaryKey} to row ID mappings which only contains + * {@link Row} regardless it's live or deleted. ({@link RangeTombstoneMarker} is not included.) + * + * For JBOD, we can make use of sstable min/max partition key to filter irrelevant {@link MemtableIndex} subranges. + * For Tiered Storage, in most cases, it flushes to tiered 0. + */ +public class RowMapping +{ + public static final RowMapping DUMMY = new RowMapping() + { + @Override + public Iterator> merge(MemtableIndex index) { return Collections.emptyIterator(); } + + @Override + public void complete() {} + + @Override + public void add(DecoratedKey key, Unfiltered unfiltered, long sstableRowId) {} + }; + + private final MemtableTrie rowMapping = new MemtableTrie<>(BufferType.OFF_HEAP); + + private volatile boolean complete = false; + + public DecoratedKey minKey; + public DecoratedKey maxKey; + + public int maxSegmentRowId = -1; + + private RowMapping() + { + } + + /** + * Create row mapping for FLUSH operation only. + */ + public static RowMapping create(OperationType opType) + { + if (opType == OperationType.FLUSH) + return new RowMapping(); + return DUMMY; + } + + /** + * Merge IndexMemtable(index term to PrimaryKeys mappings) with row mapping of a sstable + * (PrimaryKey to RowId mappings). + * + * @param index a Memtable-attached column index + * + * @return iterator of index term to postings mapping exists in the sstable + */ + public Iterator> merge(MemtableIndex index) + { + assert complete : "RowMapping is not built."; + + Iterator> iterator = index.iterator(); + return new AbstractIterator>() + { + @Override + protected Pair computeNext() + { + while (iterator.hasNext()) + { + Pair pair = iterator.next(); + + IntArrayList postings = null; + Iterator primaryKeys = pair.right.iterator(); + + while (primaryKeys.hasNext()) + { + PrimaryKey primaryKey = primaryKeys.next(); + ByteComparable byteComparable = asComparableBytes(primaryKey.partitionKey(), primaryKey.clustering()); + Integer segmentRowId = rowMapping.get(byteComparable); + + if (segmentRowId != null) + { + postings = postings == null ? new IntArrayList() : postings; + postings.add(segmentRowId); + } + } + if (postings != null && !postings.isEmpty()) + return Pair.create(pair.left, postings); + } + return endOfData(); + } + }; + } + + /** + * Complete building in memory RowMapping, mark it as immutable. + */ + public void complete() + { + assert !complete : "RowMapping can only be built once."; + this.complete = true; + } + + /** + * Include PrimaryKey to RowId mapping + */ + public void add(DecoratedKey key, Unfiltered unfiltered, long sstableRowId) + { + assert !complete : "Cannot modify built RowMapping."; + + if (unfiltered.isRangeTombstoneMarker()) + { + // currently we don't record range tombstones.. + } + else + { + assert unfiltered.isRow(); + Row row = (Row) unfiltered; + + ByteComparable byteComparable = asComparableBytes(key, row.clustering()); + int segmentRowId = SegmentBuilder.castToSegmentRowId(sstableRowId, 0); + try + { + rowMapping.apply(Trie.singleton(byteComparable, segmentRowId), (existing, neww) -> neww); + } + catch (MemtableTrie.SpaceExhaustedException e) + { + //TODO Work out how to handle this properly + throw new RuntimeException(e); + } + + maxSegmentRowId = Math.max(maxSegmentRowId, segmentRowId); + + // data is written in token sorted order + if (minKey == null) + minKey = key; + maxKey = key; + } + } + + public boolean hasRows() + { + return maxSegmentRowId >= 0; + } + + private ByteComparable asComparableBytes(DecoratedKey key, Clustering clustering) + { + return v -> new ByteSource() + { + ByteSource source = key.asComparableBytes(v); + int index = -1; + + @Override + public int next() + { + if (index == clustering.size()) + return END_OF_STREAM; + + int b = source.next(); + if (b > END_OF_STREAM) + return b; + + if (++index == clustering.size()) + return v == ByteComparable.Version.LEGACY ? ByteSource.END_OF_STREAM : ByteSource.TERMINATOR; + source = ByteSource.of(clustering.accessor(), clustering.get(index), v); + return NEXT_COMPONENT; + } + }; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java new file mode 100644 index 000000000000..e47977f8dac2 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java @@ -0,0 +1,339 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.memory; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.SortedSet; +import java.util.concurrent.atomic.LongAdder; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.util.concurrent.FastThreadLocal; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.tries.MemtableTrie; +import org.apache.cassandra.db.tries.Trie; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.analyzer.AbstractAnalyzer; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.index.sai.utils.PrimaryKeys; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; + +public class TrieMemoryIndex extends MemoryIndex +{ + private static final Logger logger = LoggerFactory.getLogger(TrieMemoryIndex.class); + private static final int MINIMUM_QUEUE_SIZE = 128; + private static final int MAX_RECURSIVE_KEY_LENGTH = 128; + + + private final MemtableTrie data; + private final ClusteringComparator clusteringComparator; + private final PrimaryKeysReducer primaryKeysReducer; + private final AbstractAnalyzer analyzer; + private final AbstractType validator; + private final boolean isLiteral; + private final Object writeLock = new Object(); + + private static final FastThreadLocal lastQueueSize = new FastThreadLocal() + { + protected Integer initialValue() + { + return MINIMUM_QUEUE_SIZE; + } + }; + + + public TrieMemoryIndex(ColumnContext columnContext) + { + super(columnContext); + //TODO Do we need to follow a setting for this? + this.data = new MemtableTrie<>(BufferType.OFF_HEAP); + this.clusteringComparator = columnContext.clusteringComparator(); + this.primaryKeysReducer = new PrimaryKeysReducer(); + // MemoryIndex is per-core, so analyzer should be thread-safe.. + this.analyzer = columnContext.getAnalyzer(); + this.validator = columnContext.getValidator(); + this.isLiteral = TypeUtil.isLiteral(validator); + } + + @Override + public long add(DecoratedKey key, Clustering clustering, ByteBuffer value) + { + synchronized (writeLock) + { + AbstractAnalyzer analyzer = columnContext.getAnalyzer(); + value = TypeUtil.encode(value, validator); + analyzer.reset(value.duplicate()); + final PrimaryKey primaryKey = PrimaryKey.of(key, clustering); + final long initialSizeOnHeap = data.sizeOnHeap(); + final long initialSizeOffHeap = data.sizeOffHeap(); + final long reducerHeapSize = primaryKeysReducer.heapAllocations(); + + + while (analyzer.hasNext()) + { + final ByteBuffer term = analyzer.next(); + setMinMaxTerm(term); + + final ByteComparable encodedTerm = encode(term); + try + { + if (term.limit() <= MAX_RECURSIVE_KEY_LENGTH) + { + data.putRecursive(encodedTerm, primaryKey, primaryKeysReducer); + } + else + { + data.apply(Trie.singleton(encodedTerm, primaryKey), primaryKeysReducer); + } + } + catch (MemtableTrie.SpaceExhaustedException e) + { + //TODO Handle this properly + throw new RuntimeException(e); + } + } + return (data.sizeOnHeap() - initialSizeOnHeap) + (data.sizeOffHeap() - initialSizeOffHeap) + (primaryKeysReducer.heapAllocations() - reducerHeapSize); + } + } + + @Override + public RangeIterator search(Expression expression, AbstractBounds keyRange) + { + if (logger.isTraceEnabled()) + logger.trace("Searching memtable index on expression '{}'...", expression); + + switch (expression.getOp()) + { + case MATCH: + case EQ: + case CONTAINS_KEY: + case CONTAINS_VALUE: + return exactMatch(expression); + case RANGE: + return rangeMatch(expression, keyRange); + default: + throw new IllegalArgumentException("Unsupported expression: " + expression); + } + } + + @Override + public Iterator> iterator() + { + Iterator> iterator = data.entrySet().iterator(); + return new Iterator>() + { + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + + @Override + public Pair next() + { + Map.Entry entry = iterator.next(); + return Pair.create(decode(entry.getKey()), entry.getValue()); + } + }; + } + + private ByteComparable encode(ByteBuffer input) + { + return isLiteral ? version -> append(ByteSource.of(input, version), ByteSource.TERMINATOR) + : version -> TypeUtil.asComparableBytes(input, validator, version); + } + + private ByteComparable decode(ByteComparable term) + { + return isLiteral ? version -> ByteSourceInverse.unescape(ByteSource.peekable(term.asComparableBytes(version))) + : term; + + } + + private ByteSource append(ByteSource src, int lastByte) + { + return new ByteSource() + { + boolean done = false; + + @Override + public int next() + { + if (done) + return END_OF_STREAM; + int n = src.next(); + if (n != END_OF_STREAM) + return n; + + done = true; + return lastByte; + } + }; + } + + private RangeIterator exactMatch(Expression expression) + { + final ByteComparable prefix = expression.lower == null ? ByteComparable.EMPTY : encode(expression.lower.value.encoded); + final PrimaryKeys primaryKeys = data.get(prefix); + if (primaryKeys == null) + { + return RangeIterator.empty(); + } + return new KeyRangeIterator(primaryKeys.partitionKeys()); + } + + public static class Collector + { + long minimumTokenValue = Long.MAX_VALUE; + long maximumTokenValue = Long.MIN_VALUE; + PriorityQueue mergedKeys = new PriorityQueue<>(lastQueueSize.get(), DecoratedKey.comparator); + + AbstractBounds keyRange; + + public Collector(AbstractBounds keyRange) + { + this.keyRange = keyRange; + } + + public void processContent(PrimaryKeys keys) + { + if (keys.isEmpty()) + return; + + SortedSet partitionKeys = keys.partitionKeys(); + + // shortcut to avoid generating iterator + if (partitionKeys.size() == 1) + { + DecoratedKey first = partitionKeys.first(); + if (keyRange.contains(first)) + { + mergedKeys.add(first); + + long currentTokenValue = first.getToken().getLongValue(); + minimumTokenValue = Math.min(minimumTokenValue, currentTokenValue); + maximumTokenValue = Math.max(maximumTokenValue, currentTokenValue); + } + + return; + } + + // skip entire partition keys if they don't overlap + if (!keyRange.right.isMinimum() && partitionKeys.first().compareTo(keyRange.right) > 0 + || partitionKeys.last().compareTo(keyRange.left) < 0) + return; + + for (DecoratedKey key : partitionKeys) + { + if (keyRange.contains(key)) + { + mergedKeys.add(key); + + long currentTokenValue = key.getToken().getLongValue(); + minimumTokenValue = Math.min(minimumTokenValue, currentTokenValue); + maximumTokenValue = Math.max(maximumTokenValue, currentTokenValue); + } + } + return; + } + } + + private RangeIterator rangeMatch(Expression expression, AbstractBounds keyRange) + { + ByteComparable lowerBound, upperBound; + boolean lowerInclusive, upperInclusive; + if (expression.lower != null) + { + lowerBound = encode(expression.lower.value.encoded); + lowerInclusive = expression.lower.inclusive; + } + else + { + lowerBound = ByteComparable.EMPTY; + lowerInclusive = false; + } + + if (expression.upper != null) + { + upperBound = encode(expression.upper.value.encoded); + upperInclusive = expression.upper.inclusive; + } + else + { + upperBound = null; + upperInclusive = false; + } + + Collector cd = new Collector(keyRange); + + data.subtrie(lowerBound, lowerInclusive, upperBound, upperInclusive).values().forEach(pk -> cd.processContent(pk)); + + if (cd.mergedKeys.isEmpty()) + { + return RangeIterator.empty(); + } + + lastQueueSize.set(Math.max(MINIMUM_QUEUE_SIZE, cd.mergedKeys.size())); + return new KeyRangeIterator(cd.minimumTokenValue, cd.maximumTokenValue, cd.mergedKeys); + } + + private class PrimaryKeysReducer implements MemtableTrie.UpsertTransformer + { + private final LongAdder heapAllocations = new LongAdder(); + + @Override + public PrimaryKeys apply(PrimaryKeys existing, PrimaryKey neww) + { + if (existing == null) + { + existing = PrimaryKeys.create(clusteringComparator); + heapAllocations.add(existing.unsharedHeapSize()); + } + heapAllocations.add(existing.add(neww)); + return existing; + } + + long heapAllocations() + { + return heapAllocations.longValue(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/AbstractMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/AbstractMetrics.java new file mode 100644 index 000000000000..b34cfdde8f2a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/AbstractMetrics.java @@ -0,0 +1,90 @@ +/* + * 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.index.sai.metrics; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.metrics.CassandraMetricsRegistry; +import org.apache.cassandra.metrics.DefaultNameFactory; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +public abstract class AbstractMetrics +{ + public static final String TYPE = "StorageAttachedIndex"; + + protected final TableMetadata table; + private final String index; + private final String scope; + protected final List tracked = new ArrayList<>(); + + AbstractMetrics(TableMetadata table, String scope) + { + this(table, null, scope); + } + + AbstractMetrics(TableMetadata table, String index, String scope) + { + assert table != null : "SAI metrics must include table metadata"; + this.table = table; + this.index = index; + this.scope = scope; + } + + public void release() + { + tracked.forEach(Metrics::remove); + tracked.clear(); + } + + protected CassandraMetricsRegistry.MetricName createMetricName(String name) + { + return createMetricName(name, scope); + } + + protected CassandraMetricsRegistry.MetricName createMetricName(String name, String scope) + { + String metricScope = table.keyspace + "." + table.name; + if (index != null) + { + metricScope += "." + index; + } + metricScope += "." + scope + "." + name; + + CassandraMetricsRegistry.MetricName metricName = new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, + TYPE, name, metricScope, createMBeanName(name, scope)); + tracked.add(metricName); + return metricName; + } + + private String createMBeanName(String name, String scope) + { + StringBuilder builder = new StringBuilder(); + builder.append(DefaultNameFactory.GROUP_NAME); + builder.append(":type=").append(TYPE); + builder.append(',').append("keyspace=").append(table.keyspace); + builder.append(',').append("table=").append(table.name); + if (index != null) + builder.append(',').append("index=").append(index); + builder.append(',').append("scope=").append(scope); + builder.append(',').append("name=").append(name); + return builder.toString(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java new file mode 100644 index 000000000000..4cbb9124ff0e --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java @@ -0,0 +1,146 @@ +/* + * 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.index.sai.metrics; + +import java.util.concurrent.TimeUnit; + +import com.codahale.metrics.Meter; +import com.codahale.metrics.Timer; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +public abstract class ColumnQueryMetrics extends AbstractMetrics +{ + private ColumnQueryMetrics(String indexName, TableMetadata table) + { + super(table, indexName, "ColumnQueryMetrics"); + } + + public static class TrieIndexMetrics extends ColumnQueryMetrics implements QueryEventListener.TrieIndexEventListener + { + private static final String TRIE_POSTINGS_TYPE = "Postings"; + + /** + * Trie index metrics. + */ + private final Timer termsTraversalTotalTime; + + private final QueryEventListener.PostingListEventListener postingsListener; + + public TrieIndexMetrics(String indexName, TableMetadata table) + { + super(indexName, table); + + termsTraversalTotalTime = Metrics.timer(createMetricName("TermsLookupLatency")); + + Meter postingDecodes = Metrics.meter(createMetricName("PostingDecodes", TRIE_POSTINGS_TYPE)); + + postingsListener = new PostingListEventsMetrics(postingDecodes); + } + + @Override + public void onSegmentHit() { } + + @Override + public void onTraversalComplete(long traversalTotalTime, TimeUnit unit) + { + termsTraversalTotalTime.update(traversalTotalTime, unit); + } + + @Override + public QueryEventListener.PostingListEventListener postingListEventListener() + { + return postingsListener; + } + } + + public static class BKDIndexMetrics extends ColumnQueryMetrics implements QueryEventListener.BKDIndexEventListener + { + private static final String BKD_POSTINGS_TYPE = "KDTreePostings"; + + /** + * BKD index metrics. + */ + private final Timer intersectionLatency; + private final Meter postingsNumPostings; + private final Meter intersectionEarlyExits; + + private final QueryEventListener.PostingListEventListener postingsListener; + + public BKDIndexMetrics(String indexName, TableMetadata table) + { + super(indexName, table); + + intersectionLatency = Metrics.timer(createMetricName("KDTreeIntersectionLatency")); + intersectionEarlyExits = Metrics.meter(createMetricName("KDTreeIntersectionEarlyExits")); + + postingsNumPostings = Metrics.meter(createMetricName("NumPostings", BKD_POSTINGS_TYPE)); + + Meter postingDecodes = Metrics.meter(createMetricName("PostingDecodes", BKD_POSTINGS_TYPE)); + + postingsListener = new PostingListEventsMetrics(postingDecodes); + } + + @Override + public void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit) + { + intersectionLatency.update(intersectionTotalTime, unit); + } + + @Override + public void onIntersectionEarlyExit() + { + intersectionEarlyExits.mark(); + } + + @Override + public void postingListsHit(int count) + { + postingsNumPostings.mark(count); + } + + @Override + public void onSegmentHit() { } + + @Override + public QueryEventListener.PostingListEventListener postingListEventListener() + { + return postingsListener; + } + } + + private static class PostingListEventsMetrics implements QueryEventListener.PostingListEventListener + { + private final Meter postingDecodes; + + private PostingListEventsMetrics(Meter postingDecodes) + { + this.postingDecodes = postingDecodes; + } + + @Override + public void onAdvance() { } + + @Override + public void postingDecoded(long postingsDecoded) + { + postingDecodes.mark(postingsDecoded); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/IndexGroupMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/IndexGroupMetrics.java new file mode 100644 index 000000000000..50ed57b92ecd --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/IndexGroupMetrics.java @@ -0,0 +1,39 @@ +/* + * 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.index.sai.metrics; + +import com.codahale.metrics.Gauge; +import org.apache.cassandra.index.sai.StorageAttachedIndexGroup; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +public class IndexGroupMetrics extends AbstractMetrics +{ + public final Gauge openIndexFiles; + public final Gauge diskUsedBytes; + + public IndexGroupMetrics(TableMetadata table, StorageAttachedIndexGroup group) + { + super(table, "IndexGroupMetrics"); + + openIndexFiles = Metrics.register(createMetricName("OpenIndexFiles"), group::openIndexFiles); + + diskUsedBytes = Metrics.register(createMetricName("DiskUsedBytes"), group::diskUsage); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/IndexMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/IndexMetrics.java new file mode 100644 index 000000000000..29881ff59740 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/IndexMetrics.java @@ -0,0 +1,68 @@ +/* + * 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.index.sai.metrics; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Timer; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +public class IndexMetrics extends AbstractMetrics +{ + public final Timer memtableIndexWriteLatency; + + public final Gauge ssTableCellCount; + public final Gauge liveMemtableIndexWriteCount; + public final Gauge diskUsedBytes; + public final Gauge memtableIndexBytes; + public final Gauge indexFileCacheBytes; + + public final Counter memtableIndexFlushCount; + public final Counter compactionCount; + public final Counter memtableIndexFlushErrors; + public final Counter segmentFlushErrors; + + public final Histogram memtableFlushCellsPerSecond; + public final Histogram segmentsPerCompaction; + public final Histogram compactionSegmentCellsPerSecond; + public final Histogram compactionSegmentBytesPerSecond; + + public IndexMetrics(ColumnContext context, TableMetadata table) + { + super(table, context.getIndexName(), "IndexMetrics"); + + memtableIndexWriteLatency = Metrics.timer(createMetricName("MemtableIndexWriteLatency")); + compactionSegmentCellsPerSecond = Metrics.histogram(createMetricName("CompactionSegmentCellsPerSecond"), false); + compactionSegmentBytesPerSecond = Metrics.histogram(createMetricName("CompactionSegmentBytesPerSecond"), false); + memtableFlushCellsPerSecond = Metrics.histogram(createMetricName("MemtableIndexFlushCellsPerSecond"), false); + segmentsPerCompaction = Metrics.histogram(createMetricName("SegmentsPerCompaction"), false); + ssTableCellCount = Metrics.register(createMetricName("SSTableCellCount"), context::getCellCount); + memtableIndexFlushCount = Metrics.counter(createMetricName("MemtableIndexFlushCount")); + compactionCount = Metrics.counter(createMetricName("CompactionCount")); + memtableIndexFlushErrors = Metrics.counter(createMetricName("MemtableIndexFlushErrors")); + segmentFlushErrors = Metrics.counter(createMetricName("CompactionSegmentFlushErrors")); + liveMemtableIndexWriteCount = Metrics.register(createMetricName("LiveMemtableIndexWriteCount"), context::liveMemtableWriteCount); + memtableIndexBytes = Metrics.register(createMetricName("MemtableIndexBytes"), context::estimatedMemIndexMemoryUsed); + diskUsedBytes = Metrics.register(createMetricName("DiskUsedBytes"), context::diskUsage); + indexFileCacheBytes = Metrics.register(createMetricName("IndexFileCacheBytes"), context::indexFileCacheSize); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java b/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java new file mode 100644 index 000000000000..2096ef0fd902 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java @@ -0,0 +1,168 @@ +/* + * 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.index.sai.metrics; + +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.index.sai.QueryContext; + +public final class MulticastQueryEventListeners +{ + public static QueryEventListener.TrieIndexEventListener of(QueryContext ctx, QueryEventListener.TrieIndexEventListener listener) + { + return new Multicast2TrieIndexEventListener(ctx, listener); + } + + public static QueryEventListener.BKDIndexEventListener of(QueryContext ctx, QueryEventListener.BKDIndexEventListener listener) + { + return new Multicast2BKDIndexEventListener(ctx, listener); + } + + public static class Multicast2TrieIndexEventListener implements QueryEventListener.TrieIndexEventListener + { + private final QueryContext ctx; + private final QueryEventListener.TrieIndexEventListener listener; + private final Multicast2TriePostingListEventListener postingListEventListener; + + private Multicast2TrieIndexEventListener(QueryContext ctx, QueryEventListener.TrieIndexEventListener listener) + { + this.ctx = ctx; + this.listener = listener; + this.postingListEventListener = new Multicast2TriePostingListEventListener(ctx, listener.postingListEventListener()); + } + + @Override + public void onSegmentHit() + { + ctx.segmentsHit++; + ctx.trieSegmentsHit++; + listener.onSegmentHit(); + } + + @Override + public void onTraversalComplete(long traversalTotalTime, TimeUnit unit) + { + listener.onTraversalComplete(traversalTotalTime, unit); + } + + @Override + public QueryEventListener.PostingListEventListener postingListEventListener() + { + return postingListEventListener; + } + } + + public static class Multicast2BKDIndexEventListener implements QueryEventListener.BKDIndexEventListener + { + private final QueryContext ctx; + private final QueryEventListener.BKDIndexEventListener listener; + private final Multicast2BKDPostingListEventListener postingListEventListener; + + private Multicast2BKDIndexEventListener(QueryContext ctx, QueryEventListener.BKDIndexEventListener listener) + { + this.ctx = ctx; + this.listener = listener; + this.postingListEventListener = new Multicast2BKDPostingListEventListener(ctx, listener.postingListEventListener()); + } + + @Override + public void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit) + { + listener.onIntersectionComplete(intersectionTotalTime, unit); + } + + @Override + public void onIntersectionEarlyExit() + { + listener.onIntersectionEarlyExit(); + } + + @Override + public void postingListsHit(int count) + { + ctx.bkdPostingListsHit++; + listener.postingListsHit(count); + } + + @Override + public void onSegmentHit() + { + ctx.segmentsHit++; + ctx.bkdSegmentsHit++; + listener.onSegmentHit(); + } + + @Override + public QueryEventListener.PostingListEventListener postingListEventListener() + { + return postingListEventListener; + } + } + + public static class Multicast2BKDPostingListEventListener implements QueryEventListener.PostingListEventListener + { + private final QueryContext ctx; + private final QueryEventListener.PostingListEventListener listener; + + Multicast2BKDPostingListEventListener(QueryContext ctx, QueryEventListener.PostingListEventListener listener) + { + this.ctx = ctx; + this.listener = listener; + } + + @Override + public void onAdvance() + { + ctx.bkdPostingsSkips++; + listener.onAdvance(); + } + + @Override + public void postingDecoded(long postingDecoded) + { + ctx.bkdPostingsDecodes += postingDecoded; + listener.postingDecoded(postingDecoded); + } + } + + public static class Multicast2TriePostingListEventListener implements QueryEventListener.PostingListEventListener + { + private final QueryContext ctx; + private final QueryEventListener.PostingListEventListener listener; + + Multicast2TriePostingListEventListener(QueryContext ctx, QueryEventListener.PostingListEventListener listener) + { + this.ctx = ctx; + this.listener = listener; + } + + @Override + public void onAdvance() + { + ctx.triePostingsSkips++; + listener.onAdvance(); + } + + @Override + public void postingDecoded(long postingDecoded) + { + ctx.triePostingsDecodes += postingDecoded; + listener.postingDecoded(postingDecoded); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java b/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java new file mode 100644 index 000000000000..4f410fa97e66 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java @@ -0,0 +1,116 @@ +/* + * 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.index.sai.metrics; + +import java.util.concurrent.TimeUnit; + +/** + * Listener that gets notified during storage-attached index query execution. + */ +public interface QueryEventListener +{ + /** + * Returns listener for bkd index events. + */ + BKDIndexEventListener bkdIndexEventListener(); + + /** + * Returns listener for trie index events. + */ + TrieIndexEventListener trieIndexEventListener(); + + /** + * Collector for kd-tree index file related metrics. + */ + interface BKDIndexEventListener + { + /** + * Per-segment kd-tree index intersection time in given units. Recorded when intersection completes. + */ + void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit); + + /** + * When an intersection exits early due to the query shape being completely outside the min/max range. + */ + void onIntersectionEarlyExit(); + + /** + * How many bkd posting list were matched during the intersection. + */ + void postingListsHit(int count); + + /** + * When query potentially matches value range within a segment and we need to do a traversal. + */ + void onSegmentHit(); + + /** + * Returns events listener for bkd postings. + */ + PostingListEventListener postingListEventListener(); + } + + interface TrieIndexEventListener + { + /** + * When query potentially matches value range within a segment and we need to do a traversal. + */ + void onSegmentHit(); + + /** + * Per-segment trie index traversal time in given units. Recorded when traversal completes. + */ + void onTraversalComplete(long traversalTotalTime, TimeUnit unit); + + /** + * Returns events listener for trie postings. + */ + PostingListEventListener postingListEventListener(); + } + + /** + * Collector for posting file related metrics. + */ + interface PostingListEventListener + { + /** + * When an individual posting lists is advanced. + */ + void onAdvance(); + + /** + * When a posting is successfully read from disk and decoded. + */ + void postingDecoded(long postingsDecoded); + + PostingListEventListener NO_OP = new PostingListEventListener() + { + @Override + public void onAdvance() + { + + } + + @Override + public void postingDecoded(long postingsDecoded) + { + + } + }; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java new file mode 100644 index 000000000000..5b38f3919f1a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java @@ -0,0 +1,199 @@ +/* + * 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.index.sai.metrics; + +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Timer; +import org.apache.cassandra.index.sai.QueryContext; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.NoSpamLogger; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +public class TableQueryMetrics extends AbstractMetrics +{ + public static final String TABLE_QUERY_METRIC_TYPE = "TableQueryMetrics"; + private static final Logger logger = LoggerFactory.getLogger(TableQueryMetrics.class); + private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 60, TimeUnit.SECONDS); + + private final PerQueryMetrics perQueryMetrics; + + private final Counter totalQueryTimeouts; + private final Counter totalPartitionReads; + private final Counter totalRowsFiltered; + private final Counter totalQueriesCompleted; + + private final Meter tokenSkippingLookups; + private final Meter tokenSkippingCacheHits; + + public TableQueryMetrics(TableMetadata table) + { + super(table, TABLE_QUERY_METRIC_TYPE); + + perQueryMetrics = new PerQueryMetrics(table); + + totalPartitionReads = Metrics.counter(createMetricName("TotalPartitionReads")); + totalRowsFiltered = Metrics.counter(createMetricName("TotalRowsFiltered")); + totalQueriesCompleted = Metrics.counter(createMetricName("TotalQueriesCompleted")); + totalQueryTimeouts = Metrics.counter(createMetricName("TotalQueryTimeouts")); + + tokenSkippingLookups = Metrics.meter(createMetricName("Lookups", "TokenSkipping")); + tokenSkippingCacheHits = Metrics.meter(createMetricName("CacheHits", "TokenSkipping")); + } + + public void record(QueryContext queryContext) + { + if (queryContext.queryTimeouts > 0) + { + assert queryContext.queryTimeouts == 1; + + totalQueryTimeouts.inc(); + } + + long skippingLookups = queryContext.tokenSkippingLookups; + long skippingCacheHits = queryContext.tokenSkippingCacheHits; + + tokenSkippingLookups.mark(skippingLookups); + tokenSkippingCacheHits.mark(skippingCacheHits); + + perQueryMetrics.record(queryContext); + } + + public void release() + { + super.release(); + perQueryMetrics.release(); + } + + public class PerQueryMetrics extends AbstractMetrics + { + private final Timer queryLatency; + + /** + * Global metrics for all indices hit during the query. + */ + private final Histogram sstablesHit; + private final Histogram segmentsHit; + private final Histogram partitionReads; + private final Histogram rowsFiltered; + + /** + * BKD index metrics. + */ + private final Histogram kdTreePostingsNumPostings; + /** + * BKD index posting lists metrics. + */ + private final Histogram kdTreePostingsSkips; + private final Histogram kdTreePostingsDecodes; + /** + * Trie index posting lists metrics. + */ + private final Histogram postingsSkips; + private final Histogram postingsDecodes; + + public PerQueryMetrics(TableMetadata table) + { + super(table, "PerQuery"); + + queryLatency = Metrics.timer(createMetricName("QueryLatency")); + + sstablesHit = Metrics.histogram(createMetricName("SSTableIndexesHit"), false); + segmentsHit = Metrics.histogram(createMetricName("IndexSegmentsHit"), false); + + kdTreePostingsSkips = Metrics.histogram(createMetricName("KDTreePostingsSkips"), false); + + kdTreePostingsNumPostings = Metrics.histogram(createMetricName("KDTreePostingsNumPostings"), false); + kdTreePostingsDecodes = Metrics.histogram(createMetricName("KDTreePostingsDecodes"), false); + + postingsSkips = Metrics.histogram(createMetricName("PostingsSkips"), false); + postingsDecodes = Metrics.histogram(createMetricName("PostingsDecodes"), false); + + partitionReads = Metrics.histogram(createMetricName("PartitionReads"), false); + rowsFiltered = Metrics.histogram(createMetricName("RowsFiltered"), false); + } + + private void recordStringIndexCacheMetrics(QueryContext events) + { + postingsSkips.update(events.triePostingsSkips); + postingsDecodes.update(events.triePostingsDecodes); + } + + private void recordNumericIndexCacheMetrics(QueryContext events) + { + kdTreePostingsNumPostings.update(events.bkdPostingListsHit); + + kdTreePostingsSkips.update(events.bkdPostingsSkips); + kdTreePostingsDecodes.update(events.bkdPostingsDecodes); + } + + public void record(QueryContext queryContext) + { + final long totalQueryTimeNs = queryContext.totalQueryTimeNs(); + queryLatency.update(totalQueryTimeNs, TimeUnit.NANOSECONDS); + final long queryLatencyMicros = TimeUnit.NANOSECONDS.toMicros(totalQueryTimeNs); + + final long ssTablesHit = queryContext.sstablesHit; + final long segmentsHit = queryContext.segmentsHit; + final long partitionsRead = queryContext.partitionsRead; + final long rowsFiltered = queryContext.rowsFiltered; + + sstablesHit.update(ssTablesHit); + this.segmentsHit.update(segmentsHit); + + partitionReads.update(partitionsRead); + totalPartitionReads.inc(partitionsRead); + + this.rowsFiltered.update(rowsFiltered); + totalRowsFiltered.inc(rowsFiltered); + + if (Tracing.isTracing()) + { + Tracing.trace("Index query accessed memtable indexes, {}, and {}, post-filtered {} in {}, and took {} microseconds.", + pluralize(ssTablesHit, "SSTable index", "es"), pluralize(segmentsHit, "segment", "s"), + pluralize(rowsFiltered, "row", "s"), pluralize(partitionsRead, "partition", "s"), + queryLatencyMicros); + } + + if (queryContext.trieSegmentsHit > 0) + { + recordStringIndexCacheMetrics(queryContext); + } + + if (queryContext.bkdSegmentsHit > 0) + { + recordNumericIndexCacheMetrics(queryContext); + } + + totalQueriesCompleted.inc(); + } + } + + private String pluralize(long count, String root, String plural) + { + return count == 1 ? String.format("1 %s", root) : String.format("%d %s%s", count, root, plural); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/TableStateMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/TableStateMetrics.java new file mode 100644 index 000000000000..9841770f730b --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/metrics/TableStateMetrics.java @@ -0,0 +1,52 @@ +/* + * 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.index.sai.metrics; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.RatioGauge; +import org.apache.cassandra.index.sai.StorageAttachedIndexGroup; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +public class TableStateMetrics extends AbstractMetrics +{ + public static final String TABLE_STATE_METRIC_TYPE = "TableStateMetrics"; + + private final Gauge diskUsageBytes; + private final Gauge diskUsagePercentageOfBaseTable; + private final Gauge totalIndexCount; + private final Gauge totalIndexBuildsInProgress; + private final Gauge totalQueryableIndexCount; + + public TableStateMetrics(TableMetadata table, StorageAttachedIndexGroup group) + { + super(table, TABLE_STATE_METRIC_TYPE); + totalQueryableIndexCount = Metrics.register(createMetricName("TotalQueryableIndexCount"), group::totalQueryableIndexCount); + totalIndexCount = Metrics.register(createMetricName("TotalIndexCount"), group::totalIndexCount); + totalIndexBuildsInProgress = Metrics.register(createMetricName("TotalIndexBuildsInProgress"), group::totalIndexBuildsInProgress); + diskUsageBytes = Metrics.register(createMetricName("DiskUsedBytes"), group::totalDiskUsage); + diskUsagePercentageOfBaseTable = Metrics.register(createMetricName("DiskPercentageOfBaseTable"), new RatioGauge() { + @Override + protected Ratio getRatio() + { + return Ratio.of(group.totalDiskUsage(), group.table().metric.liveDiskSpaceUsed.getCount()); + } + }); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/plan/Expression.java b/src/java/org/apache/cassandra/index/sai/plan/Expression.java new file mode 100644 index 000000000000..dfe0c1b7dadc --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/plan/Expression.java @@ -0,0 +1,428 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.plan; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Iterators; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.analyzer.AbstractAnalyzer; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.utils.ByteBufferUtil; + +public class Expression +{ + private static final Logger logger = LoggerFactory.getLogger(Expression.class); + + public enum Op + { + EQ, MATCH, PREFIX, NOT_EQ, RANGE, CONTAINS_KEY, CONTAINS_VALUE, IN; + + public static Op valueOf(Operator operator) + { + switch (operator) + { + case EQ: + return EQ; + + case NEQ: + return NOT_EQ; + + case CONTAINS: + return CONTAINS_VALUE; // non-frozen map: value contains term; + + case CONTAINS_KEY: + return CONTAINS_KEY; // non-frozen map: value contains key term; + + case LT: + case GT: + case LTE: + case GTE: + return RANGE; + + case LIKE_PREFIX: + return PREFIX; + + case LIKE_MATCHES: + return MATCH; + + case IN: + return IN; + + default: + return null; + } + } + + public boolean isEquality() + { + return this == EQ || this == CONTAINS_KEY || this == CONTAINS_VALUE; + } + + public boolean isEqualityOrRange() + { + return isEquality() || this == RANGE; + } + } + + public final AbstractAnalyzer analyzer; + + public final ColumnContext context; + public final AbstractType validator; + + @VisibleForTesting + protected Op operation; + + public Bound lower, upper; + public boolean upperInclusive, lowerInclusive; + + final List exclusions = new ArrayList<>(); + + public Expression(ColumnContext columnContext) + { + this.context = columnContext; + this.analyzer = columnContext.getAnalyzer(); + this.validator = columnContext.getValidator(); + } + + public Expression add(Operator op, ByteBuffer value) + { + boolean lowerInclusive, upperInclusive; + // If the type supports rounding then we need to make sure that index + // range search is always inclusive, otherwise we run the risk of + // missing values that are within the exclusive range but are rejected + // because their rounded value is the same as the value being queried. + lowerInclusive = upperInclusive = TypeUtil.supportsRounding(validator); + switch (op) + { + case LIKE_PREFIX: + case LIKE_MATCHES: + case EQ: + case CONTAINS: + case CONTAINS_KEY: + lower = new Bound(value, validator, true); + upper = lower; + operation = Op.valueOf(op); + break; + + case NEQ: + // index expressions are priority sorted + // and NOT_EQ is the lowest priority, which means that operation type + // is always going to be set before reaching it in case of RANGE or EQ. + if (operation == null) + { + operation = Op.NOT_EQ; + lower = new Bound(value, validator, true); + upper = lower; + } + else + exclusions.add(value); + break; + + case LTE: + if (context.getDefinition().isReversedType()) + { + this.lowerInclusive = true; + lowerInclusive = true; + } + else + { + this.upperInclusive = true; + upperInclusive = true; + } + case LT: + operation = Op.RANGE; + if (context.getDefinition().isReversedType()) + lower = new Bound(value, validator, lowerInclusive); + else + upper = new Bound(value, validator, upperInclusive); + break; + + case GTE: + if (context.getDefinition().isReversedType()) + { + this.upperInclusive = true; + upperInclusive = true; + } + else + { + this.lowerInclusive = true; + lowerInclusive = true; + } + case GT: + operation = Op.RANGE; + if (context.getDefinition().isReversedType()) + upper = new Bound(value, validator, upperInclusive); + else + lower = new Bound(value, validator, lowerInclusive); + break; + } + + return this; + } + + public boolean isSatisfiedBy(ByteBuffer columnValue) + { + if (!TypeUtil.isValid(columnValue, validator)) + { + logger.error(context.logMessage("Value is not valid for indexed column {} with {}"), context.getColumnName(), validator); + return false; + } + + Value value = new Value(columnValue, validator); + + if (lower != null) + { + // suffix check + if (TypeUtil.isLiteral(validator)) + { + if (!validateStringValue(value.raw, lower.value.raw)) + return false; + } + else + { + // range or (not-)equals - (mainly) for numeric values + int cmp = TypeUtil.comparePostFilter(lower.value, value, validator); + + // in case of (NOT_)EQ lower == upper + if (operation == Op.EQ || operation == Op.CONTAINS_KEY || operation == Op.CONTAINS_VALUE || operation == Op.NOT_EQ) + return cmp == 0; + + if (cmp > 0 || (cmp == 0 && !lowerInclusive)) + return false; + } + } + + if (upper != null && lower != upper) + { + // string (prefix or suffix) check + if (TypeUtil.isLiteral(validator)) + { + if (!validateStringValue(value.raw, upper.value.raw)) + return false; + } + else + { + // range - mainly for numeric values + int cmp = TypeUtil.comparePostFilter(upper.value, value, validator); + if (cmp < 0 || (cmp == 0 && !upperInclusive)) + return false; + } + } + + // as a last step let's check exclusions for the given field, + // this covers EQ/RANGE with exclusions. + for (ByteBuffer term : exclusions) + { + if (TypeUtil.isLiteral(validator) && validateStringValue(value.raw, term) || + TypeUtil.comparePostFilter(new Value(term, validator), value, validator) == 0) + return false; + } + + return true; + } + + private boolean validateStringValue(ByteBuffer columnValue, ByteBuffer requestedValue) + { + analyzer.reset(columnValue.duplicate()); + while (analyzer.hasNext()) + { + ByteBuffer term = analyzer.next(); + + boolean isMatch = false; + switch (operation) + { + case EQ: + case MATCH: + // Operation.isSatisfiedBy handles conclusion on !=, + // here we just need to make sure that term matched it + case CONTAINS_KEY: + case CONTAINS_VALUE: + case NOT_EQ: + isMatch = validator.compare(term, requestedValue) == 0; + break; + case RANGE: + isMatch = isLowerSatisfiedBy(term) && isUpperSatisfiedBy(term); + break; + + case PREFIX: + isMatch = ByteBufferUtil.startsWith(term, requestedValue); + break; + } + + if (isMatch) + return true; + } + + return false; + } + + public Op getOp() + { + return operation; + } + + private boolean hasLower() + { + return lower != null; + } + + private boolean hasUpper() + { + return upper != null; + } + + private boolean isLowerSatisfiedBy(ByteBuffer value) + { + if (!hasLower()) + return true; + + int cmp = validator.compare(value, lower.value.raw); + return cmp > 0 || cmp == 0 && lower.inclusive; + } + + private boolean isUpperSatisfiedBy(ByteBuffer value) + { + if (!hasUpper()) + return true; + + int cmp = validator.compare(value, upper.value.raw); + return cmp < 0 || cmp == 0 && upper.inclusive; + } + + public String toString() + { + return String.format("Expression{name: %s, op: %s, lower: (%s, %s), upper: (%s, %s), exclusions: %s}", + context.getColumnName(), + operation, + lower == null ? "null" : validator.getString(lower.value.raw), + lower != null && lower.inclusive, + upper == null ? "null" : validator.getString(upper.value.raw), + upper != null && upper.inclusive, + Iterators.toString(Iterators.transform(exclusions.iterator(), validator::getString))); + } + + public int hashCode() + { + return new HashCodeBuilder().append(context.getColumnName()) + .append(operation) + .append(validator) + .append(lower).append(upper) + .append(exclusions).build(); + } + + public boolean equals(Object other) + { + if (!(other instanceof Expression)) + return false; + + if (this == other) + return true; + + Expression o = (Expression) other; + + return Objects.equals(context.getColumnName(), o.context.getColumnName()) + && validator.equals(o.validator) + && operation == o.operation + && Objects.equals(lower, o.lower) + && Objects.equals(upper, o.upper) + && exclusions.equals(o.exclusions); + } + + /** + * A representation of a column value in it's raw and encoded form. + */ + public static class Value + { + public final ByteBuffer raw; + public final ByteBuffer encoded; + + public Value(ByteBuffer value, AbstractType type) + { + this.raw = value; + this.encoded = TypeUtil.encode(value, type); + } + + @Override + public boolean equals(Object other) + { + if (!(other instanceof Value)) + return false; + + Value o = (Value) other; + return raw.equals(o.raw) && encoded.equals(o.encoded); + } + + @Override + public int hashCode() + { + HashCodeBuilder builder = new HashCodeBuilder(); + builder.append(raw); + builder.append(encoded); + return builder.toHashCode(); + } + } + + public static class Bound + { + public final Value value; + public final boolean inclusive; + + public Bound(ByteBuffer value, AbstractType type, boolean inclusive) + { + this.value = new Value(value, type); + this.inclusive = inclusive; + } + + @Override + public boolean equals(Object other) + { + if (!(other instanceof Bound)) + return false; + + Bound o = (Bound) other; + return value.equals(o.value) && inclusive == o.inclusive; + } + + @Override + public int hashCode() + { + HashCodeBuilder builder = new HashCodeBuilder(); + builder.append(value); + builder.append(inclusive); + return builder.toHashCode(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/plan/FilterTree.java b/src/java/org/apache/cassandra/index/sai/plan/FilterTree.java new file mode 100644 index 000000000000..7264877cb1b8 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/plan/FilterTree.java @@ -0,0 +1,146 @@ +/* + * 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.index.sai.plan; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; + +import com.google.common.collect.ListMultimap; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.index.sai.plan.Expression.Op; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.ColumnMetadata.Kind; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.index.sai.plan.Operation.OperationType; + +/** + * Tree-like structure to filter base table data using indexed expressions and non-user-defined filters. + * + * This is needed because: + * 1. SAI doesn't index tombstones, base data may have been shadowed. + * 2. SAI indexes partition offset, not all rows in partition match index condition. + * 3. Replica filter protecting may fetch data that doesn't match index expressions. + */ +public class FilterTree +{ + protected final OperationType op; + protected final ListMultimap expressions; + protected final List children = new ArrayList<>(); + + FilterTree(OperationType operation, + ListMultimap expressions) + { + this.op = operation; + this.expressions = expressions; + } + + void addChild(FilterTree child) + { + children.add(child); + } + + public boolean isSatisfiedBy(DecoratedKey key, Unfiltered currentCluster, Row staticRow) + { + boolean result = localSatisfiedBy(key, currentCluster, staticRow); + + for (FilterTree child : children) + result = op.apply(result, child.isSatisfiedBy(key, currentCluster, staticRow)); + + return result; + } + + private boolean localSatisfiedBy(DecoratedKey key, Unfiltered currentCluster, Row staticRow) + { + if (currentCluster == null || !currentCluster.isRow()) + return false; + + final int now = FBUtilities.nowInSeconds(); + boolean result = op == OperationType.AND; + + Iterator columnIterator = expressions.keySet().iterator(); + while(columnIterator.hasNext()) + { + ColumnMetadata column = columnIterator.next(); + Row row = column.kind == Kind.STATIC ? staticRow : (Row)currentCluster; + + // If there is a column with multiple expressions that can mean an OR or (in the case of map + // collections) it can mean different map indexes. + List filters = expressions.get(column); + + // We do a reverse iteration over the filters because NOT_EQ operations will be at the end + // of the filter list and we want to check them first. + ListIterator filterIterator = filters.listIterator(filters.size()); + while(filterIterator.hasPrevious()) + { + Expression filter = filterIterator.previous(); + + if (TypeUtil.isNonFrozenCollection(column.type)) + { + Iterator valueIterator = filter.context.getValuesOf(row, now); + result = op.apply(result, collectionMatch(valueIterator, filter)); + } + else + { + ByteBuffer value = filter.context.getValueOf(key, row, now); + result = op.apply(result, singletonMatch(value, filter)); + } + + // If the operation is an AND then exit early if we get a single false + if (op == OperationType.AND && !result) + return false; + else if (op == OperationType.OR && result) + return true; + } + } + return result; + } + + private boolean singletonMatch(ByteBuffer value, Expression filter) + { + boolean match = value != null && filter.isSatisfiedBy(value); + // If this is NOT_EQ operation we have to + // inverse match flag (to check against other expressions), + if (filter.getOp() == Op.NOT_EQ) + match = !match; + return match; + } + + private boolean collectionMatch(Iterator valueIterator, Expression filter) + { + if (valueIterator == null) + return false; + + while (valueIterator.hasNext()) + { + ByteBuffer value = valueIterator.next(); + if (value == null) + continue; + if (filter.isSatisfiedBy(value)) + return true; + } + return false; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/plan/Operation.java b/src/java/org/apache/cassandra/index/sai/plan/Operation.java new file mode 100644 index 000000000000..2d4a8e52d369 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/plan/Operation.java @@ -0,0 +1,398 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.plan; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Iterables; +import com.google.common.collect.ListMultimap; + +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.analyzer.AbstractAnalyzer; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.serializers.ListSerializer; +import org.apache.cassandra.transport.ProtocolVersion; + +public class Operation +{ + public enum OperationType + { + AND, OR; + + public boolean apply(boolean a, boolean b) + { + switch (this) + { + case OR: + return a | b; + + case AND: + return a & b; + + default: + throw new AssertionError(); + } + } + } + + @VisibleForTesting + protected static ListMultimap analyzeGroup(QueryController controller, + OperationType op, + List expressions) + { + ListMultimap analyzed = ArrayListMultimap.create(); + + // sort all of the expressions in the operation by name and priority of the logical operator + // this gives us an efficient way to handle inequality and combining into ranges without extra processing + // and converting expressions from one type to another. + expressions.sort((a, b) -> { + int cmp = a.column().compareTo(b.column()); + return cmp == 0 ? -Integer.compare(getPriority(a.operator()), getPriority(b.operator())) : cmp; + }); + + for (final RowFilter.Expression e : expressions) + { + ColumnContext columnContext = controller.getContext(e); + List perColumn = analyzed.get(e.column()); + + AbstractAnalyzer analyzer = columnContext.getAnalyzer(); + analyzer.reset(e.getIndexValue().duplicate()); + + // EQ/LIKE_*/NOT_EQ can have multiple expressions e.g. text = "Hello World", + // becomes text = "Hello" OR text = "World" because "space" is always interpreted as a split point (by analyzer), + // CONTAINS/CONTAINS_KEY are always treated as multiple expressions since they currently only targetting + // collections, NOT_EQ is made an independent expression only in case of pre-existing multiple EQ expressions, or + // if there is no EQ operations and NOT_EQ is met or a single NOT_EQ expression present, + // in such case we know exactly that there would be no more EQ/RANGE expressions for given column + // since NOT_EQ has the lowest priority. + boolean isMultiExpression = false; + switch (e.operator()) + { + case EQ: + // EQ operator will always be a multiple expression because it is being used by + // map entries + isMultiExpression = columnContext.isNonFrozenCollection(); + break; + + case CONTAINS: + case CONTAINS_KEY: + case LIKE_PREFIX: + case LIKE_MATCHES: + isMultiExpression = true; + break; + + case NEQ: + isMultiExpression = (perColumn.size() == 0 || perColumn.size() > 1 + || (perColumn.size() == 1 && perColumn.get(0).getOp() == Expression.Op.NOT_EQ)); + break; + } + if (isMultiExpression) + { + while (analyzer.hasNext()) + { + final ByteBuffer token = analyzer.next(); + perColumn.add(new Expression(columnContext).add(e.operator(), token)); + } + } + else + // "range" or not-equals operator, combines both bounds together into the single expression, + // iff operation of the group is AND, otherwise we are forced to create separate expressions, + // not-equals is combined with the range iff operator is AND. + { + Expression range; + if (perColumn.size() == 0 || op != OperationType.AND) + { + perColumn.add((range = new Expression(columnContext))); + } + else + { + range = Iterables.getLast(perColumn); + } + + if (!TypeUtil.isLiteral(columnContext.getValidator())) + { + range.add(e.operator(), e.getIndexValue().duplicate()); + } + else + { + while (analyzer.hasNext()) + { + range.add(e.operator(), analyzer.next()); + } + } + } + } + + return analyzed; + } + + private static int getPriority(org.apache.cassandra.cql3.Operator op) + { + switch (op) + { + case EQ: + case CONTAINS: + case CONTAINS_KEY: + return 5; + + case LIKE_PREFIX: + case LIKE_MATCHES: + return 4; + + case GTE: + case GT: + return 3; + + case LTE: + case LT: + return 2; + + case NEQ: + return 1; + + default: + return 0; + } + } + + static RangeIterator buildIterator(QueryController controller) + { + return Node.buildTree(controller.filterOperation()).analyzeTree(controller).rangeIterator(controller); + } + + static FilterTree buildFilter(QueryController controller) + { + return Node.buildTree(controller.filterOperation()).buildFilter(controller); + } + + public static abstract class Node + { + ListMultimap expressionMap; + + boolean canFilter() + { + return (expressionMap != null && !expressionMap.isEmpty()) || !children().isEmpty() ; + } + + List children() + { + return Collections.emptyList(); + } + + void add(Node child) + { + throw new UnsupportedOperationException(); + } + + RowFilter.Expression expression() + { + throw new UnsupportedOperationException(); + } + + abstract void analyze(List expressionList, QueryController controller); + + abstract FilterTree filterTree(); + + abstract RangeIterator rangeIterator(QueryController controller); + + static Node buildTree(RowFilter.FilterElement filterOperation) + { + OperatorNode node = filterOperation.isDisjunction() ? new OrNode() : new AndNode(); + for (RowFilter.Expression expression : filterOperation.expressions()) + node.add(buildExpression(expression)); + for (RowFilter.FilterElement child : filterOperation.children()) + node.add(buildTree(child)); + return node; + } + + static Node buildExpression(RowFilter.Expression expression) + { + if (expression.operator() == Operator.IN) + { + OperatorNode node = new OrNode(); + int size = ListSerializer.readCollectionSize(expression.getIndexValue(), ByteBufferAccessor.instance, ProtocolVersion.V3); + int offset = ListSerializer.sizeOfCollectionSize(size, ProtocolVersion.V3); + for (int index = 0; index < size; index++) + { + node.add(new ExpressionNode(new RowFilter.SimpleExpression(expression.column(), + Operator.EQ, + ListSerializer.readValue(expression.getIndexValue(), + ByteBufferAccessor.instance, + offset, + ProtocolVersion.V3)))); + offset += TypeSizes.INT_SIZE + ByteBufferAccessor.instance.getInt(expression.getIndexValue(), offset); + } + return node; + } + else + return new ExpressionNode(expression); + } + + Node analyzeTree(QueryController controller) + { + List expressionList = new ArrayList<>(); + doTreeAnalysis(this, expressionList, controller); + if (!expressionList.isEmpty()) + this.analyze(expressionList, controller); + return this; + } + + void doTreeAnalysis(Node node, List expressions, QueryController controller) + { + if (node.children().isEmpty()) + expressions.add(node.expression()); + else + { + List expressionList = new ArrayList<>(); + for (Node child : node.children()) + doTreeAnalysis(child, expressionList, controller); + node.analyze(expressionList, controller); + } + } + + FilterTree buildFilter(QueryController controller) + { + analyzeTree(controller); + FilterTree tree = filterTree(); + for (Node child : children()) + if (child.canFilter()) + tree.addChild(child.buildFilter(controller)); + return tree; + } + } + + public static abstract class OperatorNode extends Node + { + List children = new ArrayList<>(); + + @Override + public List children() + { + return children; + } + + @Override + public void add(Node child) + { + children.add(child); + } + } + + public static class AndNode extends OperatorNode + { + @Override + public void analyze(List expressionList, QueryController controller) + { + expressionMap = analyzeGroup(controller, OperationType.AND, expressionList); + } + + @Override + FilterTree filterTree() + { + return new FilterTree(OperationType.AND, expressionMap); + } + + @Override + RangeIterator rangeIterator(QueryController controller) + { + RangeIterator.Builder builder = controller.getIndexes(OperationType.AND, expressionMap.values()); + for (Node child : children) + if (child.canFilter()) + builder.add(child.rangeIterator(controller)); + return builder.build(); + } + } + + public static class OrNode extends OperatorNode + { + @Override + public void analyze(List expressionList, QueryController controller) + { + expressionMap = analyzeGroup(controller, OperationType.OR, expressionList); + } + + @Override + FilterTree filterTree() + { + return new FilterTree(OperationType.OR, expressionMap); + } + + @Override + RangeIterator rangeIterator(QueryController controller) + { + RangeIterator.Builder builder = controller.getIndexes(OperationType.OR, expressionMap.values()); + for (Node child : children) + if (child.canFilter()) + builder.add(child.rangeIterator(controller)); + return builder.build(); + } + } + + public static class ExpressionNode extends Node + { + RowFilter.Expression expression; + + @Override + public void analyze(List expressionList, QueryController controller) + { + expressionMap = analyzeGroup(controller, OperationType.AND, expressionList); + } + + @Override + FilterTree filterTree() + { + return new FilterTree(OperationType.AND, expressionMap); + } + + public ExpressionNode(RowFilter.Expression expression) + { + this.expression = expression; + } + + @Override + public RowFilter.Expression expression() + { + return expression; + } + + @Override + RangeIterator rangeIterator(QueryController controller) + { + assert canFilter(); + return controller.getIndexes(OperationType.AND, expressionMap.values()).build(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java new file mode 100644 index 000000000000..b0cfad2527cb --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java @@ -0,0 +1,380 @@ +/* + * 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.index.sai.plan; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.MultiRangeReadCommand; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.PartitionRangeReadCommand; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.SinglePartitionReadCommand; +import org.apache.cassandra.db.filter.DataLimits; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.QueryContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.index.sai.metrics.TableQueryMetrics; +import org.apache.cassandra.index.sai.utils.RangeIntersectionIterator; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.index.sai.utils.RangeUnionIterator; +import org.apache.cassandra.index.sai.utils.TermIterator; +import org.apache.cassandra.index.sai.view.View; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.concurrent.Ref; + +public class QueryController +{ + private static final Logger logger = LoggerFactory.getLogger(QueryController.class); + + private final ColumnFamilyStore cfs; + private final ReadCommand command; + private final QueryContext queryContext; + private final TableQueryMetrics tableQueryMetrics; + private final RowFilter.FilterElement filterOperation; + + private final List ranges; + private final AbstractBounds mergeRange; + + public QueryController(ColumnFamilyStore cfs, + ReadCommand command, + RowFilter.FilterElement filterOperation, + QueryContext queryContext, + TableQueryMetrics tableQueryMetrics) + { + this.cfs = cfs; + this.command = command; + this.queryContext = queryContext; + this.tableQueryMetrics = tableQueryMetrics; + this.filterOperation = filterOperation; + this.ranges = dataRanges(command); + DataRange first = ranges.get(0); + DataRange last = ranges.get(ranges.size() - 1); + this.mergeRange = ranges.size() == 1 ? first.keyRange() : first.keyRange().withNewRight(last.keyRange().right); + } + + public TableMetadata metadata() + { + return command.metadata(); + } + + RowFilter.FilterElement filterOperation() + { + return this.filterOperation; + } + + /** + * @return token ranges used in the read command + */ + List dataRanges() + { + return ranges; + } + + /** + * Note: merged range may contain subrange that no longer belongs to the local node after range movement. + * It should only be used as an optimization to reduce search space. Use {@link #dataRanges()} instead to filter data. + * + * @return merged token range + */ + AbstractBounds mergeRange() + { + return mergeRange; + } + + /** + * @return indexed {@code ColumnContext} if index is found; otherwise return non-indexed {@code ColumnContext}. + */ + public ColumnContext getContext(RowFilter.Expression expression) + { + StorageAttachedIndex index = getBestIndexFor(expression); + + return index != null ? index.getContext() : new ColumnContext(cfs.metadata(), expression.column()); + } + + public StorageAttachedIndex getBestIndexFor(RowFilter.Expression expression) + { + return cfs.indexManager.getBestIndexFor(expression, StorageAttachedIndex.class).orElse(null); + } + + public UnfilteredRowIterator getPartition(DecoratedKey key, ReadExecutionController executionController) + { + if (key == null) + throw new IllegalArgumentException("non-null key required"); + + try + { + SinglePartitionReadCommand partition = SinglePartitionReadCommand.create(cfs.metadata(), + command.nowInSec(), + command.columnFilter(), + RowFilter.NONE, + DataLimits.NONE, + key, + command.clusteringIndexFilter(key)); + + return partition.queryMemtableAndDisk(cfs, executionController); + } + finally + { + queryContext.checkpoint(); + } + } + + /** + * Build a {@link RangeIterator.Builder} from the given list of expressions by applying given operation (OR/AND). + * Building of such builder involves index search, results of which are persisted in the internal resources list + * + * @param op The operation type to coalesce expressions with. + * @param expressions The expressions to build range iterator from (expressions with not results are ignored). + * + * @return range iterator builder based on given expressions and operation type. + */ + public RangeIterator.Builder getIndexes(Operation.OperationType op, Collection expressions) + { + boolean defer = op == Operation.OperationType.OR || RangeIntersectionIterator.shouldDefer(expressions.size()); + + RangeIterator.Builder builder = op == Operation.OperationType.OR + ? RangeUnionIterator.builder() + : RangeIntersectionIterator.selectiveBuilder(); + + Set>> view = referenceAndGetView(op, expressions).entrySet(); + + try + { + for (Map.Entry> e : view) + { + @SuppressWarnings("resource") // RangeIterators are closed by releaseIndexes + RangeIterator index = TermIterator.build(e.getKey(), e.getValue(), mergeRange, queryContext, defer); + + builder.add(index); + } + } + catch (Throwable t) + { + // all sstable indexes in view have been referenced, need to clean up when exception is thrown + FileUtils.closeQuietly(builder.ranges()); + view.forEach(e -> e.getValue().forEach(SSTableIndex::release)); + throw t; + } + return builder; + } + + private static void releaseQuietly(SSTableIndex index) + { + try + { + index.release(); + } + catch (Throwable e) + { + logger.error(index.getColumnContext().logMessage("Failed to release index on SSTable {}"), index.getSSTable().descriptor, e); + } + } + + /** + * Used to release all resources and record metrics when query finishes. + */ + public void finish() + { + if (tableQueryMetrics != null) tableQueryMetrics.record(queryContext); + } + + /** + * Try to reference all SSTableIndexes before querying on disk indexes. + * + * If we attempt to proceed into {@link TermIterator#build(Expression, Set, AbstractBounds, QueryContext, boolean)} + * without first referencing all indexes, a concurrent compaction may decrement one or more of their backing + * SSTable {@link Ref} instances. This will allow the {@link SSTableIndex} itself to be released and will fail the query. + */ + private Map> referenceAndGetView(Operation.OperationType op, Collection expressions) + { + SortedSet indexNames = new TreeSet<>(); + try + { + while (true) + { + List referencedIndexes = new ArrayList<>(); + boolean failed = false; + + Map> view = getView(op, expressions); + + for (SSTableIndex index : view.values().stream().flatMap(Collection::stream).collect(Collectors.toList())) + { + indexNames.add(index.getColumnContext().getIndexName()); + + if (index.reference()) + { + referencedIndexes.add(index); + } + else + { + failed = true; + break; + } + } + + if (failed) + { + // TODO: This might be a good candidate for a table/index group metric in the future... + referencedIndexes.forEach(QueryController::releaseQuietly); + } + else + { + return view; + } + } + } + finally + { + Tracing.trace("Querying storage-attached indexes {}", indexNames); + } + } + + private Map> getView(Operation.OperationType op, Collection expressions) + { + // first let's determine the primary expression if op is AND + Pair> primary = (op == Operation.OperationType.AND) ? calculatePrimary(expressions) : null; + + Map> indexes = new HashMap<>(); + for (Expression e : expressions) + { + // NO_EQ and non-index column query should only act as FILTER BY for satisfiedBy(Row) method + // because otherwise it likely to go through the whole index. + if (!e.context.isIndexed() || e.getOp() == Expression.Op.NOT_EQ) + { + continue; + } + + // primary expression, we'll have to add as is + if (primary != null && e.equals(primary.left)) + { + indexes.put(primary.left, primary.right); + + continue; + } + + View view = e.context.getView(); + + NavigableSet readers = new TreeSet<>(SSTableIndex.COMPARATOR); + if (primary != null && primary.right.size() > 0) + { + for (SSTableIndex index : primary.right) + readers.addAll(view.match(index.minKey(), index.maxKey())); + } + else + { + readers.addAll(applyScope(view.match(e))); + } + + indexes.put(e, readers); + } + + return indexes; + } + + private Pair> calculatePrimary(Collection expressions) + { + Expression expression = null; + NavigableSet primaryIndexes = null; + + for (Expression e : expressions) + { + if (!e.context.isIndexed()) + continue; + + View view = e.context.getView(); + + NavigableSet indexes = new TreeSet<>(SSTableIndex.COMPARATOR); + indexes.addAll(applyScope(view.match(e))); + + if (expression == null || primaryIndexes.size() > indexes.size()) + { + primaryIndexes = indexes; + expression = e; + } + } + + return expression == null ? null : Pair.create(expression, primaryIndexes); + } + + private Set applyScope(Set indexes) + { + return Sets.filter(indexes, index -> { + SSTableReader sstable = index.getSSTable(); + + return mergeRange.left.compareTo(sstable.last) <= 0 && (mergeRange.right.isMinimum() || sstable.first.compareTo(mergeRange.right) <= 0); + }); + } + + /** + * Returns the {@link DataRange} list covered by the specified {@link ReadCommand}. + * + * @param command a read command + * @return the data ranges covered by {@code command} + */ + private static List dataRanges(ReadCommand command) + { + if (command instanceof SinglePartitionReadCommand) + { + SinglePartitionReadCommand cmd = (SinglePartitionReadCommand) command; + DecoratedKey key = cmd.partitionKey(); + return Lists.newArrayList(new DataRange(new Range<>(key, key), cmd.clusteringIndexFilter())); + } + else if (command instanceof PartitionRangeReadCommand) + { + PartitionRangeReadCommand cmd = (PartitionRangeReadCommand) command; + return Lists.newArrayList(cmd.dataRange()); + } + else if (command instanceof MultiRangeReadCommand) + { + MultiRangeReadCommand cmd = (MultiRangeReadCommand) command; + return cmd.ranges(); + } + else + { + throw new AssertionError("Unsupported read command type: " + command.getClass().getName()); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexQueryPlan.java b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexQueryPlan.java new file mode 100644 index 000000000000..47e970e1ae5e --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexQueryPlan.java @@ -0,0 +1,141 @@ +/* + * 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.index.sai.plan; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.index.sai.metrics.TableQueryMetrics; +import org.apache.cassandra.schema.TableMetadata; + +public class StorageAttachedIndexQueryPlan implements Index.QueryPlan +{ + private final ColumnFamilyStore cfs; + private final TableQueryMetrics queryMetrics; + private final RowFilter postIndexFilter; + private final RowFilter.FilterElement filterOperation; + private final Set indexes; + + private StorageAttachedIndexQueryPlan(ColumnFamilyStore cfs, + TableQueryMetrics queryMetrics, + RowFilter postIndexFilter, + RowFilter.FilterElement filterOperation, + ImmutableSet indexes) + { + this.cfs = cfs; + this.queryMetrics = queryMetrics; + this.postIndexFilter = postIndexFilter; + this.filterOperation = filterOperation; + this.indexes = indexes; + } + + @Nullable + public static StorageAttachedIndexQueryPlan create(ColumnFamilyStore cfs, + TableQueryMetrics queryMetrics, + Set indexes, + RowFilter rowFilter) + { + ImmutableSet.Builder selectedIndexesBuilder = ImmutableSet.builder(); + List acceptedExpressions = new ArrayList<>(); + + for (RowFilter.Expression expression : rowFilter) + { + // we ignore user-defined expressions here because we don't have a way to translate their #isSatifiedBy + // method, they will be included in the filter returned by QueryPlan#postIndexQueryFilter() + if (expression.isUserDefined()) + continue; + + acceptedExpressions.add(expression); + for (StorageAttachedIndex index : indexes) + { + if (index.supportsExpression(expression.column(), expression.operator())) + { + selectedIndexesBuilder.add(index); + } + } + } + + ImmutableSet selectedIndexes = selectedIndexesBuilder.build(); + if (selectedIndexes.isEmpty()) + return null; + + /* + * postIndexFilter comprised by those expressions in the read command row filter that can't be handled by + * {@link FilterTree#satisfiedBy(Unfiltered, Row, boolean)}. That includes expressions targeted + * at {@link RowFilter.UserExpression}s like those used by RLAC. + */ + RowFilter postIndexFilter = rowFilter.restrict(e -> e.isUserDefined()); + return new StorageAttachedIndexQueryPlan(cfs, queryMetrics, postIndexFilter, rowFilter.root(), selectedIndexes); + } + + @Override + public Set getIndexes() + { + return indexes; + } + + @Override + public long getEstimatedResultRows() + { + // this is temporary (until proper QueryPlan is integrated into Cassandra) + // and allows us to priority storage-attached indexes if any in the query since they + // are going to be more efficient, to query and intersect, than built-in indexes. + return Long.MIN_VALUE; + } + + @Override + public boolean shouldEstimateInitialConcurrency() + { + return false; + } + + @Override + public Index.Searcher searcherFor(ReadCommand command) + { + return new StorageAttachedIndexSearcher(cfs, queryMetrics, command, filterOperation, DatabaseDescriptor.getRangeRpcTimeout(TimeUnit.MILLISECONDS)); + } + + /** + * @return a filter with all the expressions that are user-defined or for a non-indexed partition key column + * + * (currently index on partition columns is not supported, see {@link StorageAttachedIndex#validateOptions(Map, TableMetadata)}) + */ + @Override + public RowFilter postIndexQueryFilter() + { + return postIndexFilter; + } + + @Override + public boolean supportsMultiRangeReadCommand() + { + return true; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java new file mode 100644 index 000000000000..33bc4fd4dda2 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java @@ -0,0 +1,394 @@ +/* + * 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.index.sai.plan; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import com.google.common.collect.Iterators; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.partitions.PartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.RowIterator; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.exceptions.RequestTimeoutException; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.sai.QueryContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.metrics.TableQueryMetrics; +import org.apache.cassandra.index.sai.utils.RangeIterator; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.AbstractIterator; + +public class StorageAttachedIndexSearcher implements Index.Searcher +{ + private final ReadCommand command; + private final QueryController controller; + private final QueryContext queryContext; + + public StorageAttachedIndexSearcher(ColumnFamilyStore cfs, + TableQueryMetrics tableQueryMetrics, + ReadCommand command, + RowFilter.FilterElement filterOperation, + long executionQuotaMs) + { + this.command = command; + this.queryContext = new QueryContext(executionQuotaMs); + this.controller = new QueryController(cfs, command, filterOperation, queryContext, tableQueryMetrics); + } + + @Override + public ReadCommand command() + { + return command; + } + + @Override + public PartitionIterator filterReplicaFilteringProtection(PartitionIterator fullResponse) + { + for (RowFilter.Expression expression : controller.filterOperation()) + { + if (controller.getContext(expression).getAnalyzer().transformValue()) + return applyIndexFilter(fullResponse, analyzeFilter(), queryContext); + } + + // if no analyzer does transformation + return Index.Searcher.super.filterReplicaFilteringProtection(fullResponse); + } + + @Override + public UnfilteredPartitionIterator search(ReadExecutionController executionController) throws RequestTimeoutException + { + return new ResultRetriever(analyze(), analyzeFilter(), controller, executionController, queryContext); + } + + /** + * Converts expressions into filter tree and reference {@link SSTableIndex}s used for query. + * + * @return operation + */ + private RangeIterator analyze() + { + return Operation.buildIterator(controller); + } + + /** + * Converts expressions into filter tree (which is currently just a single AND). + * + * Filter tree allows us to do a couple of important optimizations + * namely, group flattening for AND operations (query rewrite), expression bounds checks, + * "satisfies by" checks for resulting rows with an early exit. + * + * @return root of the filter tree. + */ + private FilterTree analyzeFilter() + { + return Operation.buildFilter(controller); + } + + private static class ResultRetriever extends AbstractIterator implements UnfilteredPartitionIterator + { + private final PartitionPosition startToken; + private final PartitionPosition lastToken; + private final Iterator keyRanges; + private AbstractBounds current; + + private final RangeIterator operation; + private final FilterTree filterTree; + private final QueryController controller; + private final ReadExecutionController executionController; + private final QueryContext queryContext; + + private Iterator currentKeys = null; + private DecoratedKey lastKey; + + private ResultRetriever(RangeIterator operation, FilterTree filterTree, QueryController controller, + ReadExecutionController executionController, QueryContext queryContext) + { + this.keyRanges = controller.dataRanges().iterator(); + this.current = keyRanges.next().keyRange(); + + this.operation = operation; + this.filterTree = filterTree; + this.controller = controller; + this.executionController = executionController; + this.queryContext = queryContext; + + this.startToken = controller.mergeRange().left; + this.lastToken = controller.mergeRange().right; + } + + @Override + public UnfilteredRowIterator computeNext() + { + if (operation == null) + return endOfData(); + + operation.skipTo(startToken.getToken().getLongValue()); + if (!operation.hasNext()) + return endOfData(); + currentKeys = operation.next().keys(); + + + // IMPORTANT: The correctness of the entire query pipeline relies on the fact that we consume a token + // and materialize its keys before moving on to the next token in the flow. This sequence must not be broken + // with toList() or similar. (Both the union and intersection flow constructs, to avoid excessive object + // allocation, reuse their token mergers as they process individual positions on the ring.) + while (true) + { + while (currentKeys.hasNext()) + { + DecoratedKey key = currentKeys.next(); + + if (!lastToken.isMinimum() && lastToken.compareTo(key) < 0) + return endOfData(); + + while (current != null) + { + if (current.contains(key)) + { + UnfilteredRowIterator partition = apply(key); + if (partition != null) + return partition; + break; + } + // bigger than current range + else if (!current.right.isMinimum() && current.right.compareTo(key) <= 0) + { + if (keyRanges.hasNext()) + current = keyRanges.next().keyRange(); + else + return endOfData(); + } + // smaller than current range + else + { + // we already knew that key is not included in "current" abstract bounds, + // so "left" may have the same partition position as "key" when "left" is exclusive. + assert current.left.compareTo(key) >= 0; + operation.skipTo(current.left.getToken().getLongValue()); + if (!operation.hasNext()) + return endOfData(); + currentKeys = operation.next().keys(); + break; + } + } + } + if (!operation.hasNext()) + return endOfData(); + currentKeys = operation.next().keys(); + } + } + + public UnfilteredRowIterator apply(DecoratedKey key) + { + // Key reads are lazy, delayed all the way to this point. Skip if we've already seen this one: + if (key.equals(lastKey)) + return null; + + lastKey = key; + + // SPRC should only return UnfilteredRowIterator, but it returns UnfilteredPartitionIterator due to Flow. + try (UnfilteredRowIterator partition = controller.getPartition(key, executionController)) + { + queryContext.partitionsRead++; + + return applyIndexFilter(key, partition, filterTree, queryContext); + } + } + + private static UnfilteredRowIterator applyIndexFilter(DecoratedKey key, UnfilteredRowIterator partition, FilterTree tree, QueryContext queryContext) + { + Row staticRow = partition.staticRow(); + List clusters = new ArrayList<>(); + + while (partition.hasNext()) + { + Unfiltered row = partition.next(); + + queryContext.rowsFiltered++; + if (tree.isSatisfiedBy(key, row, staticRow)) + clusters.add(row); + } + + if (clusters.isEmpty()) + { + queryContext.rowsFiltered++; + if (tree.isSatisfiedBy(key, staticRow, staticRow)) + clusters.add(staticRow); + } + + /* + * If {@code clusters} is empty, which means either all clustering row and static row pairs failed, + * or static row and static row pair failed. In both cases, we should not return any partition. + * If {@code clusters} is not empty, which means either there are some clustering row and static row pairs match the filters, + * or static row and static row pair matches the filters. In both cases, we should return a partition with static row, + * and remove the static row marker from the {@code clusters} for the latter case. + */ + if (clusters.isEmpty()) + return null; + + return new PartitionIterator(partition, staticRow, Iterators.filter(clusters.iterator(), u -> !((Row)u).isStatic())); + } + + private static class PartitionIterator extends AbstractUnfilteredRowIterator + { + private final Iterator rows; + + public PartitionIterator(UnfilteredRowIterator partition, Row staticRow, Iterator content) + { + super(partition.metadata(), + partition.partitionKey(), + partition.partitionLevelDeletion(), + partition.columns(), + staticRow, + partition.isReverseOrder(), + partition.stats()); + + rows = content; + } + + @Override + protected Unfiltered computeNext() + { + return rows.hasNext() ? rows.next() : endOfData(); + } + } + + @Override + public TableMetadata metadata() + { + return controller.metadata(); + } + + public void close() + { + FileUtils.closeQuietly(operation); + controller.finish(); + } + } + + /** + * Used by {@link StorageAttachedIndexSearcher#filterReplicaFilteringProtection} to filter rows for columns that + * have transformations so won't get handled correctly by the row filter. + */ + @SuppressWarnings("resource") + private static PartitionIterator applyIndexFilter(PartitionIterator response, FilterTree tree, QueryContext queryContext) + { + return new PartitionIterator() + { + @Override + public void close() + { + response.close(); + } + + @Override + public boolean hasNext() + { + return response.hasNext(); + } + + @Override + public RowIterator next() + { + RowIterator delegate = response.next(); + Row staticRow = delegate.staticRow(); + + return new RowIterator() + { + boolean hasNext; + Row next; + + @Override + public TableMetadata metadata() + { + return delegate.metadata(); + } + + @Override + public boolean isReverseOrder() + { + return delegate.isReverseOrder(); + } + + @Override + public RegularAndStaticColumns columns() + { + return delegate.columns(); + } + + @Override + public DecoratedKey partitionKey() + { + return delegate.partitionKey(); + } + + @Override + public Row staticRow() + { + return staticRow; + } + + @Override + public void close() + { + delegate.close(); + } + + @Override + public boolean hasNext() + { + while (hasNext = delegate.hasNext()) + { + next = delegate.next(); + queryContext.rowsFiltered++; + if (tree.isSatisfiedBy(delegate.partitionKey(), next, staticRow)) + return true; + } + return false; + } + + @Override + public Row next() + { + if (!hasNext) + throw new NoSuchElementException(); + return next; + } + }; + } + }; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/AbortedOperationException.java b/src/java/org/apache/cassandra/index/sai/utils/AbortedOperationException.java new file mode 100644 index 000000000000..072110808721 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/AbortedOperationException.java @@ -0,0 +1,36 @@ +/* + * 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.index.sai.utils; + + +/** + * This exception indicates that a request was aborted, normally because it was taking too much time. + * + * It is handled in a special way by the verb handlers and the request execute method: it is simply + * passed to the onAborted callback without logging any message. Therefore if any logging is required, + * it is up to the code raising this exception to log anything. + */ +// TODO OSS doesn't support onAbort and timeout response +public class AbortedOperationException extends RuntimeException +{ + public AbortedOperationException() + { + super(); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/index/sai/utils/AbstractIterator.java b/src/java/org/apache/cassandra/index/sai/utils/AbstractIterator.java new file mode 100644 index 000000000000..3285d7b748de --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/AbstractIterator.java @@ -0,0 +1,161 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed 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.index.sai.utils; + +import java.util.NoSuchElementException; + +import com.google.common.collect.PeekingIterator; + +import static com.google.common.base.Preconditions.checkState; + +// This is fork of the Guava AbstractIterator, the only difference +// is that state & next variables are now protected, this was required +// for SkippableIterator.skipTo(..) to void all previous state. +public abstract class AbstractIterator implements PeekingIterator +{ + protected State state = State.NOT_READY; + + /** Constructor for use by subclasses. */ + protected AbstractIterator() {} + + protected enum State + { + /** We have computed the next element and haven't returned it yet. */ + READY, + + /** We haven't yet computed or have already returned the element. */ + NOT_READY, + + /** We have reached the end of the data and are finished. */ + DONE, + + /** We've suffered an exception and are kaput. */ + FAILED, + } + + protected T next; + + /** + * Returns the next element. Note: the implementation must call {@link + * #endOfData()} when there are no elements left in the iteration. Failure to + * do so could result in an infinite loop. + * + *

    The initial invocation of {@link #hasNext()} or {@link #next()} calls + * this method, as does the first invocation of {@code hasNext} or {@code + * next} following each successful call to {@code next}. Once the + * implementation either invokes {@code endOfData} or throws an exception, + * {@code computeNext} is guaranteed to never be called again. + * + *

    If this method throws an exception, it will propagate outward to the + * {@code hasNext} or {@code next} invocation that invoked this method. Any + * further attempts to use the iterator will result in an {@link + * IllegalStateException}. + * + *

    The implementation of this method may not invoke the {@code hasNext}, + * {@code next}, or {@link #peek()} methods on this instance; if it does, an + * {@code IllegalStateException} will result. + * + * @return the next element if there was one. If {@code endOfData} was called + * during execution, the return value will be ignored. + * @throws RuntimeException if any unrecoverable error happens. This exception + * will propagate outward to the {@code hasNext()}, {@code next()}, or + * {@code peek()} invocation that invoked this method. Any further + * attempts to use the iterator will result in an + * {@link IllegalStateException}. + */ + protected abstract T computeNext(); + + /** + * Implementations of {@link #computeNext} must invoke this method when + * there are no elements left in the iteration. + * + * @return {@code null}; a convenience so your {@code computeNext} + * implementation can use the simple statement {@code return endOfData();} + */ + protected final T endOfData() + { + state = State.DONE; + return null; + } + + public final boolean hasNext() + { + checkState(state != State.FAILED); + + switch (state) + { + case DONE: + return false; + + case READY: + return true; + + default: + } + + return tryToComputeNext(); + } + + protected boolean tryToComputeNext() + { + state = State.FAILED; // temporary pessimism + next = computeNext(); + + if (state != State.DONE) + { + state = State.READY; + return true; + } + + return false; + } + + public final T next() + { + if (!hasNext()) + throw new NoSuchElementException(); + + state = State.NOT_READY; + return next; + } + + public void remove() + { + throw new UnsupportedOperationException(); + } + + /** + * Returns the next element in the iteration without advancing the iteration, + * according to the contract of {@link PeekingIterator#peek()}. + * + *

    Implementations of {@code AbstractIterator} that wish to expose this + * functionality should implement {@code PeekingIterator}. + */ + public final T peek() + { + if (!hasNext()) + throw new NoSuchElementException(); + + return next; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/LongArray.java b/src/java/org/apache/cassandra/index/sai/utils/LongArray.java new file mode 100644 index 000000000000..f26b75249271 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/LongArray.java @@ -0,0 +1,127 @@ +/* + * 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.index.sai.utils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.function.Supplier; + +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.disk.v1.BlockPackedReader; + +/** + * Abstraction over a long-indexed array of longs. + */ +public interface LongArray extends Closeable +{ + /** + * Get value at {@code idx}. + */ + long get(long idx); + + /** + * Get array length. + */ + long length(); + + /** + * Using the given token returns the first row ID corresponding to the token. + * @param targetToken Token to lookup and it must not be smaller than previous value + * @return The row ID of the given token or negative value if target token is greater than all tokens + */ + long findTokenRowID(long targetToken); + + @Override + default void close() throws IOException { } + + class DeferredLongArray implements LongArray + { + private Supplier supplier; + private LongArray longArray; + private boolean opened = false; + + public DeferredLongArray(Supplier supplier) + { + this.supplier = supplier; + } + + @Override + public long get(long idx) + { + open(); + return longArray.get(idx); + } + + @Override + public long length() + { + open(); + return longArray.length(); + } + + @Override + public long findTokenRowID(long targetToken) + { + open(); + return longArray.findTokenRowID(targetToken); + } + + @Override + public void close() throws IOException + { + if (opened) + longArray.close(); + } + + private void open() + { + if (!opened) + { + longArray = supplier.get(); + opened = true; + } + } + } + + interface Factory + { + LongArray open(); + + /** + * TODO use a different interface for {@link BlockPackedReader}, as {@link LongArray#findTokenRowID(long)} is + * not supported by other implementation. + * + * @param startingIndex minimum index to be used in {@link LongArray#findTokenRowID(long)}. + * In {@link org.apache.cassandra.index.sai.disk.PostingListRangeIterator}, a segmentRowId + * is provided and then in {@link OffsetFactory}, + * segment offset is applied to segmentRowId to create sstableRowId which will be used by + * {@link BlockPackedReader#openTokenReader(long, SSTableQueryContext)}. + * @param context shared between indexed columns for the same sstable in a given query + * @return token BlockPackedReader + */ + default LongArray openTokenReader(long startingIndex, SSTableQueryContext context) + { + return open(); + } + + default Factory withOffset(long idxOffset) + { + return new OffsetFactory(this, idxOffset); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java b/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java new file mode 100644 index 000000000000..4ffa6d059f4a --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java @@ -0,0 +1,79 @@ +/* + * 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.index.sai.utils; + +import java.util.concurrent.atomic.AtomicLong; +import javax.annotation.concurrent.ThreadSafe; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.utils.FBUtilities; + +/** + * A simple, thread-safe memory usage tracker, named to reflect a particular scope. + */ +@ThreadSafe +public final class NamedMemoryLimiter +{ + private static final Logger logger = LoggerFactory.getLogger(NamedMemoryLimiter.class); + + private final long limitBytes; + private final AtomicLong bytesUsed = new AtomicLong(0); + private final String scope; + + public NamedMemoryLimiter(long limitBytes, String scope) + { + this.limitBytes = limitBytes; + this.scope = scope; + + logger.info("[{}]: Memory limiter using limit of {}...", scope, FBUtilities.prettyPrintMemory(limitBytes)); + } + + /** + * @return true if the current number of bytes allocated against the tracker has breached the limit, false otherwise + */ + public boolean usageExceedsLimit() + { + return currentBytesUsed() > limitBytes; + } + + public long increment(long bytes) + { + if (logger.isTraceEnabled()) + logger.trace("[{}]: Incrementing tracked memory usage by {} bytes from current usage of {}...", scope, bytes, currentBytesUsed()); + return this.bytesUsed.addAndGet(bytes); + } + + public long decrement(long bytes) + { + if (logger.isTraceEnabled()) + logger.trace("[{}]: Decrementing tracked memory usage by {} bytes from current usage of {}...", scope, bytes, currentBytesUsed()); + return this.bytesUsed.addAndGet(-bytes); + } + + public long currentBytesUsed() + { + return this.bytesUsed.get(); + } + + public long limitBytes() + { + return this.limitBytes; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/OffsetFactory.java b/src/java/org/apache/cassandra/index/sai/utils/OffsetFactory.java new file mode 100644 index 000000000000..a2840705e416 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/OffsetFactory.java @@ -0,0 +1,157 @@ +/* + * 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.index.sai.utils; + +import java.io.IOException; + +import org.apache.cassandra.index.sai.SSTableQueryContext; + +public class OffsetFactory implements LongArray.Factory +{ + private final LongArray.Factory wrapped; + private final long idxOffset; + + OffsetFactory(LongArray.Factory wrapped, long idxOffset) + { + this.wrapped = wrapped; + this.idxOffset = idxOffset; + } + + @Override + public LongArray open() + { + return new OffsetLongArray(wrapped.open(), idxOffset); + } + + @Override + @SuppressWarnings("resource") + public LongArray openTokenReader(long segmentRowId, SSTableQueryContext context) + { + // apply segment offset so that `LongArray.findTokenRowId` will start searching tokens from current segment. + LongArray reader = wrapped.openTokenReader(segmentRowId + idxOffset, context); + return new TokenLongArray(context, reader, idxOffset); + } + + private static class OffsetLongArray implements LongArray + { + private final LongArray wrapped; + protected final long idxOffset; + + OffsetLongArray(LongArray wrapped, long idxOffset) + { + this.wrapped = wrapped; + this.idxOffset = idxOffset; + } + + /** + * Get value at {@code idx}. + */ + @Override + public long get(long idx) + { + return wrapped.get(toSSTableRowId(idx)); + } + + @Override + public long findTokenRowID(long value) + { + // Subtract the segment offset from the global row ID to provide a segment row ID for the caller: + return toSegmentRowId(wrapped.findTokenRowID(value)); + } + + /** + * Get array length. + */ + public long length() + { + return wrapped.length(); + } + + @Override + public void close() throws IOException + { + wrapped.close(); + } + + protected long toSSTableRowId(long segmentRowId) + { + return segmentRowId + idxOffset; + } + + protected long toSegmentRowId(long sstableRowId) + { + return sstableRowId - idxOffset; + } + } + + /** + * Cache the prev token value and prev sstable row id pair, and share it between different indexed columns in the + * same query. + */ + static class TokenLongArray extends OffsetLongArray + { + private final SSTableQueryContext context; + + TokenLongArray(SSTableQueryContext context, LongArray wrapped, long idxOffset) + { + super(wrapped, idxOffset); + this.context = context; + } + + @Override + public long get(long segmentRowId) + { + long sstableRowId = toSSTableRowId(segmentRowId); + if (sstableRowId == context.prevSSTableRowId) + { + return context.prevTokenValue; + } + + long tokenValue = super.get(segmentRowId); + + // during intersection, the next pair of token and rowId from current indexed column iterator is very likely + // to be used to skip another indexed column iterator (aka. used to call findTokenRowId) or used to fetch + // token (aka. get) if there is matching row id from posting reader. + context.prevTokenValue = tokenValue; + context.prevSSTableRowId = sstableRowId; + + return tokenValue; + } + + @Override + public long findTokenRowID(long tokenValue) + { + long segmentRowId = toSegmentRowId(context.prevSkipToSSTableRowId); + + // Don't use cached value from previous segment when there is duplicated tokens across segments. + if (tokenValue == context.prevSkipToTokenValue && segmentRowId >= 0) + { + context.markTokenSkippingCacheHit(); + } + else + { + segmentRowId = super.findTokenRowID(tokenValue); + + context.prevSkipToTokenValue = tokenValue; + context.prevSkipToSSTableRowId = toSSTableRowId(segmentRowId); + } + context.markTokenSkippingLookup(); + return segmentRowId; + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java new file mode 100644 index 000000000000..bb56ee38cd00 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java @@ -0,0 +1,172 @@ +/* + * 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.index.sai.utils; + +import java.util.Arrays; +import java.util.stream.Collectors; + +import com.google.common.base.Objects; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * The primary key of a row, composed by the partition key and the clustering key. + */ +public abstract class PrimaryKey +{ + private final DecoratedKey partitionKey; + + /** + * Returns a new primary key composed by the specified partition and clustering keys. + * + * @param partitionKey a partition key + * @param clustering a clustering key + * @return a new primary key composed by {@code partitionKey} and {@code ClusteringKey} + */ + public static PrimaryKey of(DecoratedKey partitionKey, Clustering clustering) + { + if (clustering == Clustering.EMPTY) + return new Skinny(partitionKey); + else if (clustering == Clustering.STATIC_CLUSTERING) + return new Static(partitionKey); + else + return new Wide(partitionKey, clustering); + } + + private PrimaryKey(DecoratedKey partitionKey) + { + this.partitionKey = partitionKey; + } + + /** + * Returns the {@link Token} of the partition key. + * + * @return the partitioning token of the partition key + */ + public Token token() + { + return partitionKey.getToken(); + } + + /** + * Returns the partition key. + * + * @return the partition key + */ + public DecoratedKey partitionKey() + { + return partitionKey; + } + + /** + * Returns the clustering key. + * + * @return the clustering key + */ + public abstract Clustering clustering(); + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + PrimaryKey that = (PrimaryKey) o; + return Objects.equal(partitionKey, that.partitionKey) && Objects.equal(clustering(), that.clustering()); + } + + @Override + public int hashCode() + { + return Objects.hashCode(partitionKey, clustering()); + } + + public String toString(TableMetadata metadata) + { + return String.format("PrimaryKey: { partition : %s, clustering: %s}", + metadata.partitionKeyType.getString(partitionKey.getKey()), + clustering().toString(metadata)); + } + + @Override + public String toString() + { + return String.format("PrimaryKey: { partition : %s, clustering: %s} "+getClass().getSimpleName(), + ByteBufferUtil.bytesToHex(partitionKey.getKey()), + String.join(",", Arrays.stream(clustering().getBufferArray()) + .map(ByteBufferUtil::bytesToHex) + .collect(Collectors.toList()))); + } + + /** + * {@link PrimaryKey} implementation for rows in tables without a defined clustering key. + */ + static class Skinny extends PrimaryKey + { + Skinny(DecoratedKey partitionKey) + { + super(partitionKey); + } + + @Override + public Clustering clustering() + { + return Clustering.EMPTY; + } + } + + /** + * {@link PrimaryKey} implementation for static rows in tables with a defined clustering key. + */ + static class Static extends PrimaryKey + { + Static(DecoratedKey partitionKey) + { + super(partitionKey); + } + + @Override + public Clustering clustering() + { + return Clustering.STATIC_CLUSTERING; + } + } + + /** + * {@link PrimaryKey} implementation for regular rows in tables with a defined clustering key. + */ + static class Wide extends PrimaryKey + { + private final Clustering clustering; + + Wide(DecoratedKey partitionKey, Clustering clustering) + { + super(partitionKey); + this.clustering = clustering; + } + + @Override + public Clustering clustering() + { + return clustering; + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKeys.java b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKeys.java new file mode 100644 index 000000000000..8f4b9e2ce6f2 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKeys.java @@ -0,0 +1,226 @@ +/* + * 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.index.sai.utils; + +import java.util.Iterator; +import java.util.Set; +import java.util.SortedSet; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.stream.Stream; +import javax.annotation.concurrent.ThreadSafe; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.utils.ObjectSizes; + +/** + * A sorted set of {@link PrimaryKey}s. + * + * The primary keys are sorted first by token, then by partition key value, and then by clustering. + */ +public interface PrimaryKeys extends Iterable +{ + // from https://github.com/gaul/java-collection-overhead + long SET_ENTRY_OVERHEAD = 36; + long MAP_ENTRY_OVERHEAD = 36; + + /** + * Returns a new empty {@link PrimaryKey} sorted set using the specified clustering comparator. + * + * @param clusteringComparator a clustering comparator + * @return a new empty primary key set + */ + static PrimaryKeys create(ClusteringComparator clusteringComparator) + { + return clusteringComparator.size() == 0 ? new Skinny() : new Wide(clusteringComparator); + } + + /** + * Adds the specified {@link PrimaryKey}. + * + * @param key a primary key + */ + default long add(PrimaryKey key) + { + return add(key.partitionKey(), key.clustering()); + } + + /** + * Adds the primary key defined by the specified partition key and clustering. + * + * @param key a partition key + * @param clustering a clustering key + */ + long add(DecoratedKey key, Clustering clustering); + + /** + * Returns all the partition keys. + * + * @return all the partition keys + */ + SortedSet partitionKeys(); + + /** + * Returns the number of primary keys. + * + * @return the number of primary keys + */ + int size(); + + /** + * Returns if this primary key set is empty. + * + * @return {@code true} if this is empty, {@code false} otherwise + */ + boolean isEmpty(); + + Stream stream(); + + long unsharedHeapSize(); + + @Override + @SuppressWarnings("NullableProblems") + default Iterator iterator() + { + return stream().iterator(); + } + + /** + * A {@link PrimaryKeys} implementation for tables without a defined clustering key, + * in which case the clustering key is always {@link Clustering#EMPTY}. + */ + @ThreadSafe + class Skinny implements PrimaryKeys + { + private static final long EMPTY_SIZE = ObjectSizes.measure(new Skinny()); + + public final ConcurrentSkipListSet keys; + + private Skinny() + { + this.keys = new ConcurrentSkipListSet<>(); + } + + @Override + public long add(DecoratedKey key, Clustering clustering) + { + assert clustering.isEmpty() : "Expected Clustering.EMPTY but got " + clustering; + return keys.add(key) ? SET_ENTRY_OVERHEAD : 0; + } + + @Override + public SortedSet partitionKeys() + { + return keys; + } + + @Override + public int size() + { + return keys.size(); + } + + @Override + public boolean isEmpty() + { + return keys.isEmpty(); + } + + @Override + public Stream stream() + { + return keys.stream().map(PrimaryKey.Skinny::new); + } + + @Override + public long unsharedHeapSize() + { + return EMPTY_SIZE; + } + } + + /** + * A {@link PrimaryKeys} implementation for tables with a defined clustering key. + */ + @ThreadSafe + class Wide implements PrimaryKeys + { + private static final long EMPTY_SIZE = ObjectSizes.measure(new Wide(null)); + + private final ClusteringComparator clusteringComparator; + private final ConcurrentSkipListMap> keys; + + private Wide(ClusteringComparator clusteringComparator) + { + this.clusteringComparator = clusteringComparator; + this.keys = new ConcurrentSkipListMap<>(); + } + + @Override + public long add(DecoratedKey key, Clustering clustering) + { + long onHeapOverhead = 0; + ConcurrentSkipListSet keys = this.keys.get(key); + + if (keys == null) + { + ConcurrentSkipListSet newKeys = new ConcurrentSkipListSet<>(clusteringComparator); + keys = this.keys.putIfAbsent(key, newKeys); + if (keys == null) + { + onHeapOverhead += (ObjectSizes.measure(newKeys) + MAP_ENTRY_OVERHEAD); + keys = newKeys; + } + } + + return keys.add(clustering) ? onHeapOverhead + SET_ENTRY_OVERHEAD : onHeapOverhead; + } + + @Override + public SortedSet partitionKeys() + { + return keys.keySet(); + } + + @Override + public int size() + { + return keys.values().stream().mapToInt(Set::size).sum(); + } + + @Override + public boolean isEmpty() + { + return keys.isEmpty(); + } + + @Override + public Stream stream() + { + return keys.entrySet().stream().flatMap(e -> e.getValue().stream().map(c -> PrimaryKey.of(e.getKey(), c))); + } + + @Override + public long unsharedHeapSize() + { + return EMPTY_SIZE; + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/RangeConcatIterator.java b/src/java/org/apache/cassandra/index/sai/utils/RangeConcatIterator.java new file mode 100644 index 000000000000..37d7546e58a4 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/RangeConcatIterator.java @@ -0,0 +1,128 @@ +/* + * 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.index.sai.utils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.PriorityQueue; + +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.io.util.FileUtils; + +/** + * {@link RangeConcatIterator} takes a list of sorted range iterator and concatenates them, leaving duplicates in + * place, to produce a new stably sorted iterator. Duplicates are eliminated later in + * {@link org.apache.cassandra.index.sai.plan.StorageAttachedIndexSearcher} + * as results from multiple SSTable indexes and their respective segments are consumed. + * + * ex. (1, 2, 3) + (3, 3, 4, 5) -> (1, 2, 3, 3, 3, 4, 5) + * ex. (1, 2, 2, 3) + (3, 4, 4, 6, 6, 7) -> (1, 2, 2, 3, 3, 4, 4, 6, 6, 7) + * + */ +public class RangeConcatIterator extends RangeIterator +{ + private final PriorityQueue ranges; + private final List toRelease; + + protected RangeConcatIterator(RangeIterator.Builder.Statistics statistics, PriorityQueue ranges) + { + super(statistics); + + this.ranges = ranges; + this.toRelease = new ArrayList<>(ranges); + } + + @Override + @SuppressWarnings("resource") + protected void performSkipTo(Long nextToken) + { + while (!ranges.isEmpty()) + { + if (ranges.peek().getCurrent().compareTo(nextToken) >= 0) + break; + + RangeIterator head = ranges.poll(); + + if (head.getMaximum().compareTo(nextToken) >= 0) + { + head.skipTo(nextToken); + ranges.add(head); + break; + } + } + } + + @Override + public void close() throws IOException + { + // due to lazy key fetching, we cannot close iterator immediately + toRelease.forEach(FileUtils::closeQuietly); + } + + @Override + @SuppressWarnings("resource") + protected Token computeNext() + { + while (!ranges.isEmpty()) + { + RangeIterator current = ranges.poll(); + if (current.hasNext()) + { + Token next = current.next(); + // hasNext will update RangeIterator's current which is used to sort in PQ + if (current.hasNext()) + ranges.add(current); + + return next; + } + } + + return endOfData(); + } + + public static Builder builder() + { + return new Builder(); + } + + public static RangeIterator build(List tokens) + { + return new Builder().add(tokens).build(); + } + + public static class Builder extends RangeIterator.Builder + { + public Builder() + { + super(IteratorType.CONCAT); + } + + protected RangeIterator buildIterator() + { + switch (rangeCount()) + { + case 1: + return ranges.poll(); + + default: + return new RangeConcatIterator(statistics, ranges); + } + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/RangeIntersectionIterator.java b/src/java/org/apache/cassandra/index/sai/utils/RangeIntersectionIterator.java new file mode 100644 index 000000000000..ba9e5b305cf8 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/RangeIntersectionIterator.java @@ -0,0 +1,258 @@ +/* + * 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.index.sai.utils; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.tracing.Tracing; + +/** + * Modified from {@link org.apache.cassandra.index.sasi.utils.RangeIntersectionIterator} to support: + * 1. no generic type to reduce allocation + * 2. support selective intersection to reduce disk io + * 3. make sure iterators are closed when intersection ends because of lazy key fetching + */ +@SuppressWarnings("resource") +public class RangeIntersectionIterator +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + // The cassandra.sai.intersection.clause.limit (default: 2) controls the maximum number of range iterator that + // will be used in the final intersection of a query operation. + private static final int INTERSECTION_CLAUSE_LIMIT = Integer.getInteger("cassandra.sai.intersection.clause.limit", 2); + + static + { + logger.info(String.format("Storage attached index intersection clause limit is %d", INTERSECTION_CLAUSE_LIMIT)); + } + + public static boolean shouldDefer(int numberOfExpressions) + { + return (INTERSECTION_CLAUSE_LIMIT <= 0) || (numberOfExpressions <= INTERSECTION_CLAUSE_LIMIT); + } + + public static Builder builder() + { + return new Builder(); + } + + public static Builder selectiveBuilder() + { + return selectiveBuilder(INTERSECTION_CLAUSE_LIMIT); + } + + public static Builder selectiveBuilder(int limit) + { + return new Builder(limit); + } + + public static class Builder extends RangeIterator.Builder + { + private final int limit; + + public Builder() + { + super(IteratorType.INTERSECTION); + this.limit = Integer.MAX_VALUE; + } + + public Builder(int limit) + { + super(IteratorType.INTERSECTION); + this.limit = limit; + } + + protected RangeIterator buildIterator() + { + // all ranges will be included + if (limit >= ranges.size() || limit <= 0) + return buildIterator(statistics, ranges); + + // Apply most selective iterators during intersection, because larger number of iterators will result lots of disk seek. + List selectiveIterator = new ArrayList<>(ranges); + selectiveIterator.sort(Comparator.comparingLong(RangeIterator::getCount)); + + Statistics selectiveStatistics = new Statistics(IteratorType.INTERSECTION); + for (int i = selectiveIterator.size() - 1; i >= 0 && i >= limit; i--) + FileUtils.closeQuietly(selectiveIterator.remove(i)); + + for (RangeIterator iterator : selectiveIterator) + selectiveStatistics.update(iterator); + + if (Tracing.isTracing()) + Tracing.trace("Selecting {} {} of {} out of {} indexes", + selectiveIterator.size(), + selectiveIterator.size() > 1 ? "indexes with cardinalities" : "index with cardinality", + selectiveIterator.stream().map(RangeIterator::getCount).map(Object::toString).collect(Collectors.joining(", ")), + ranges.size()); + + PriorityQueue selectiveRanges = new PriorityQueue<>(limit, Comparator.comparing(RangeIterator::getCurrent)); + selectiveRanges.addAll(selectiveIterator); + + return buildIterator(selectiveStatistics, selectiveRanges); + } + + private static RangeIterator buildIterator(Statistics statistics, PriorityQueue ranges) + { + // if the range is disjoint or we have an intersection with an empty set, + // we can simply return an empty iterator, because it's not going to produce any results. + if (statistics.isDisjoint()) + { + // release posting lists + FileUtils.closeQuietly(ranges); + return RangeIterator.empty(); + } + + if (ranges.size() == 1) + return ranges.poll(); + + return new BounceIntersectionIterator(statistics, ranges); + } + } + + /** + * Iterator which performs intersection of multiple ranges by using bouncing (merge-join) technique to identify + * common elements in the given ranges. Aforementioned "bounce" works as follows: range queue is poll'ed for the + * range with the smallest current token (main loop), that token is used to {@link RangeIterator#skipTo(Long)} + * other ranges, if token produced by {@link RangeIterator#skipTo(Long)} is equal to current "candidate" token, + * both get merged together and the same operation is repeated for next range from the queue, if returned token + * is not equal than candidate, candidate's range gets put back into the queue and the main loop gets repeated until + * next intersection token is found or at least one iterator runs out of tokens. + * + * This technique is every efficient to jump over gaps in the ranges. + */ + private static class BounceIntersectionIterator extends RangeIterator + { + private final PriorityQueue ranges; + private final Token.TokenMerger merger; + private final List toRelease; + private final List processedRanges; + + private BounceIntersectionIterator(Builder.Statistics statistics, PriorityQueue ranges) + { + super(statistics); + this.ranges = ranges; + this.toRelease = new ArrayList<>(ranges); + this.merger = new Token.ReusableTokenMerger(ranges.size()); + this.processedRanges = new ArrayList<>(ranges.size()); + } + + protected Token computeNext() + { + RangeIterator head = ranges.poll(); + + if (head == null) + return endOfData(); + + // jump right to the beginning of the intersection or return next element + if (head.getCurrent().compareTo(getMinimum()) < 0) + head.skipTo(getMinimum()); + + Token candidate = head.hasNext() ? head.next() : null; + + if (candidate == null || candidate.get() > getMaximum()) + return endOfData(); + + merger.reset(); + merger.add(candidate); + + processedRanges.clear(); + + boolean intersectsAll = true; + while (!ranges.isEmpty()) + { + RangeIterator range = ranges.poll(); + + // found a range which doesn't overlap with one (or possibly more) other range(s) + // or the range is exhausted + if (!isOverlapping(head, range) || (range.skipTo(candidate.get()) == null)) + { + intersectsAll = false; + break; + } + + int cmp = Long.compare(candidate.get(), range.getCurrent()); + + if (cmp == 0) + { + merger.add(range.next()); + // advance skipped range to the next element if any + range.hasNext(); + processedRanges.add(range); + } + else if (cmp < 0) + { + // the candidate is less than the current value in the next range + // so make the next range the candidate and start again + candidate = range.next(); + merger.reset(); + merger.add(candidate); + ranges.add(head); + ranges.addAll(processedRanges); + processedRanges.clear(); + head = range; + } + else + { + intersectsAll = false; + break; + } + } + + if (intersectsAll) + { + ranges.add(head); + ranges.addAll(processedRanges); + return merger.merge(); + } + + return endOfData(); + } + + protected void performSkipTo(Long nextToken) + { + List skipped = new ArrayList<>(); + + while (!ranges.isEmpty()) + { + RangeIterator range = ranges.poll(); + range.skipTo(nextToken); + skipped.add(range); + } + + for (RangeIterator range : skipped) + ranges.add(range); + } + + public void close() throws IOException + { + toRelease.forEach(FileUtils::closeQuietly); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/RangeIterator.java b/src/java/org/apache/cassandra/index/sai/utils/RangeIterator.java new file mode 100644 index 000000000000..33abd6bff9fb --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/RangeIterator.java @@ -0,0 +1,370 @@ +/* + * 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.index.sai.utils; + +import java.io.Closeable; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.io.util.FileUtils; + +/** + * Modified from {@link org.apache.cassandra.index.sasi.utils.RangeIterator} to support: + * 1. no generic type to reduce allocation + * 2. CONCAT iterator type + */ +public abstract class RangeIterator extends AbstractIterator implements Closeable +{ + private static final Builder.EmptyRangeIterator EMPTY = new Builder.EmptyRangeIterator(); + + private final Long min, max; + private final long count; + private Long current; + + protected RangeIterator(Builder.Statistics statistics) + { + this(statistics.min, statistics.max, statistics.tokenCount); + } + + public RangeIterator(RangeIterator range) + { + this(range == null ? null : range.min, range == null ? null : range.max, range == null ? -1 : range.count); + } + + public RangeIterator(Long min, Long max, long count) + { + if (min == null || max == null || count == 0) + assert min == null && max == null && (count == 0 || count == -1) : min + " - " + max + " " + count; + + this.min = min; + this.current = min; + this.max = max; + this.count = count; + } + + public final Long getMinimum() + { + return min; + } + + public final Long getCurrent() + { + return current; + } + + public final Long getMaximum() + { + return max; + } + + public final long getCount() + { + return count; + } + + /** + * When called, this iterators current position should + * be skipped forwards until finding either: + * 1) an element equal to or bigger than next + * 2) the end of the iterator + * + * @param nextToken value to skip the iterator forward until matching + * + * @return The next current token after the skip was performed + */ + public final Token skipTo(Long nextToken) + { + if (min == null || max == null) + return endOfData(); + + // In the case of deferred iterators the current value may not accurately + // reflect the next value so we need to check that as well + if (current.compareTo(nextToken) >= 0) + { + next = next == null ? recomputeNext() : next; + if (next == null) + return endOfData(); + else if (next.get().compareTo(nextToken) >= 0) + return next; + } + + if (max.compareTo(nextToken) < 0) + return endOfData(); + + performSkipTo(nextToken); + return recomputeNext(); + } + + protected abstract void performSkipTo(Long nextToken); + + protected Token recomputeNext() + { + return tryToComputeNext() ? peek() : endOfData(); + } + + protected boolean tryToComputeNext() + { + boolean hasNext = super.tryToComputeNext(); + current = hasNext ? next.get() : getMaximum(); + return hasNext; + } + + public static RangeIterator empty() + { + return EMPTY; + } + + public static abstract class Builder + { + public enum IteratorType + { + CONCAT, + UNION, + INTERSECTION; + } + + @VisibleForTesting + protected final Statistics statistics; + + @VisibleForTesting + protected final PriorityQueue ranges; + + public Builder(IteratorType type) + { + statistics = new Statistics(type); + ranges = new PriorityQueue<>(16, Comparator.comparingLong(RangeIterator::getCurrent)); + } + + public Long getMinimum() + { + return statistics.min; + } + + public Long getMaximum() + { + return statistics.max; + } + + public long getTokenCount() + { + return statistics.tokenCount; + } + + public int rangeCount() + { + return ranges.size(); + } + + public Collection ranges() + { + return ranges; + } + + public Builder add(RangeIterator range) + { + if (range == null) + return this; + + if (range.getCount() > 0) + ranges.add(range); + else + FileUtils.closeQuietly(range); + statistics.update(range); + + return this; + } + + public Builder add(List ranges) + { + if (ranges == null || ranges.isEmpty()) + return this; + + ranges.forEach(this::add); + return this; + } + + public final RangeIterator build() + { + if (rangeCount() == 0) + return empty(); + else + return buildIterator(); + } + + public static class EmptyRangeIterator extends RangeIterator + { + EmptyRangeIterator() { super(null, null, 0); } + public Token computeNext() { return endOfData(); } + protected void performSkipTo(Long nextToken) { } + public void close() { } + } + + protected abstract RangeIterator buildIterator(); + + public static class Statistics + { + protected final IteratorType iteratorType; + + protected Long min, max; + protected long tokenCount; + + // iterator with the least number of items + protected RangeIterator minRange; + // iterator with the most number of items + protected RangeIterator maxRange; + + // tracks if all of the added ranges overlap, which is useful in case of intersection, + // as it gives direct answer as to such iterator is going to produce any results. + private boolean isOverlapping = true; + + public Statistics(IteratorType iteratorType) + { + this.iteratorType = iteratorType; + } + + /** + * Update statistics information with the given range. + * + * Updates min/max of the combined range, token count and + * tracks range with the least/most number of tokens. + * + * @param range The range to update statistics with. + */ + public void update(RangeIterator range) + { + switch (iteratorType) + { + case CONCAT: + // range iterators should be sorted, but previous max must not be greater than next min. + if (range.getCount() > 0) + { + if (tokenCount == 0) + { + min = range.getMinimum(); + } + else if (tokenCount > 0 && max > range.getMinimum()) + { + throw new IllegalArgumentException("RangeIterator must be sorted, previous max: " + max + ", next min: " + range.getMinimum()); + } + + max = range.getMaximum(); + } + break; + + case UNION: + min = nullSafeMin(min, range.getMinimum()); + max = nullSafeMax(max, range.getMaximum()); + break; + + case INTERSECTION: + // minimum of the intersection is the biggest minimum of individual iterators + min = nullSafeMax(min, range.getMinimum()); + // maximum of the intersection is the smallest maximum of individual iterators + max = nullSafeMin(max, range.getMaximum()); + break; + + default: + throw new IllegalStateException("Unknown iterator type: " + iteratorType); + } + + // check if new range is disjoint with already added ranges, which means that this intersection + // is not going to produce any results, so we can cleanup range storage and never added anything to it. + isOverlapping &= isOverlapping(min, max, range); + + minRange = minRange == null ? range : min(minRange, range); + maxRange = maxRange == null ? range : max(maxRange, range); + + tokenCount += range.getCount(); + } + + private RangeIterator min(RangeIterator a, RangeIterator b) + { + return a.getCount() > b.getCount() ? b : a; + } + + private RangeIterator max(RangeIterator a, RangeIterator b) + { + return a.getCount() > b.getCount() ? a : b; + } + + public boolean isDisjoint() + { + return !isOverlapping; + } + + public double sizeRatio() + { + return minRange.getCount() * 1d / maxRange.getCount(); + } + } + } + + @VisibleForTesting + protected static boolean isOverlapping(RangeIterator a, RangeIterator b) + { + return isOverlapping(a.getCurrent(), a.getMaximum(), b); + } + + /** + * Ranges are overlapping the following cases: + * + * * When they have a common subrange: + * + * min b.current max b.max + * +---------|--------------+------------| + * + * b.current min max b.max + * |--------------+---------+------------| + * + * min b.current b.max max + * +----------|-------------|------------+ + * + * + * If either range is empty, they're disjoint. + */ + @VisibleForTesting + protected static boolean isOverlapping(Long min, Long max, RangeIterator b) + { + return (min != null && max != null) && + b.getCount() != 0 && + (min.compareTo(b.getMaximum()) <= 0 && b.getCurrent().compareTo(max) <= 0); + } + + @SuppressWarnings("unchecked") + private static T nullSafeMin(T a, T b) + { + if (a == null) return b; + if (b == null) return a; + + return a.compareTo(b) > 0 ? b : a; + } + + @SuppressWarnings("unchecked") + private static T nullSafeMax(T a, T b) + { + if (a == null) return b; + if (b == null) return a; + + return a.compareTo(b) > 0 ? a : b; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/RangeUnionIterator.java b/src/java/org/apache/cassandra/index/sai/utils/RangeUnionIterator.java new file mode 100644 index 000000000000..84686e8b815e --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/RangeUnionIterator.java @@ -0,0 +1,213 @@ +/* + * 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.index.sai.utils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.PriorityQueue; + +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.io.util.FileUtils; + +/** + * Range Union Iterator is used to return sorted stream of elements from multiple RangeIterator instances. + * + * PriorityQueue is used as a sorting mechanism for the ranges, where each computeNext() operation would poll + * from the queue (and push when done), which returns range that contains the smallest element, because + * sorting is done on the moving window of range iteration {@link RangeIterator#getCurrent()}. Once retrieved + * the smallest element (return candidate) is attempted to be merged with other ranges, because there could + * be equal elements in adjacent ranges, such ranges are poll'ed only if their {@link RangeIterator#getCurrent()} + * equals to the return candidate. + * + * Modified from {@link org.apache.cassandra.index.sasi.utils.RangeUnionIterator} to support: + * 1. no generic type to reduce allocation= + * 2. make sure iterators are closed when intersection ends because of lazy key fetching + */ +@SuppressWarnings("resource") +public class RangeUnionIterator extends RangeIterator +{ + // Due to lazy key fetching, we cannot close iterator immediately + private final PriorityQueue ranges; + + // If the ranges are deferred then the ranges queue is not + // necessarily in order so we need to maintain a separate queue + // of candidate tokens until the ranges queue is ordered correctly + private final PriorityQueue candidates; + + private final Token.TokenMerger merger; + private final List toRelease; + + private RangeUnionIterator(Builder.Statistics statistics, PriorityQueue ranges) + { + super(statistics); + this.ranges = ranges; + // Don't use Comparator.comparing here, it auto-boxes the longs + this.candidates = new PriorityQueue<>(ranges.size(), (t1, t2) -> Long.compare(t1.getLong(), t2.getLong())); + this.merger = new Token.ReusableTokenMerger(ranges.size()); + this.toRelease = new ArrayList<>(ranges); + } + + public Token computeNext() + { + Token candidate; + List processedRanges = new ArrayList<>(ranges.size()); + + // Only poll the ranges for a new candidate if the candidates queue is empty. + // Otherwise, always start with a candidate from the candidates queue until + // it is empty. + if (candidates.isEmpty()) + { + RangeIterator head = null; + + while (!ranges.isEmpty()) + { + head = ranges.poll(); + if (head.hasNext()) + break; + } + + if (head == null || !head.hasNext()) + return endOfData(); + + candidate = head.next(); + + if (head.hasNext()) + processedRanges.add(head); + } + else + { + candidate = candidates.poll(); + // may have duplicates in the candidates queue so flush them out before continuing + while (!candidates.isEmpty()) + { + if (candidate.get() < candidates.peek().get()) + break; + candidates.poll(); + } + } + + merger.reset(); + merger.add(candidate); + + long minCurrent = ranges.stream().mapToLong(RangeIterator::getCurrent).min().orElse(Long.MAX_VALUE); + + if (candidate.get() < minCurrent) + { + ranges.addAll(processedRanges); + return merger.merge(); + } + + while (!ranges.isEmpty()) + { + RangeIterator range = ranges.poll(); + + if (!range.hasNext()) + continue; + + int cmp = Long.compare(candidate.get(), range.getCurrent()); + + if (cmp == 0) + { + // If the next token is the same then consume and merge it + merger.add(range.next()); + } + else if (cmp > 0) + { + candidates.add(candidate); + candidate = range.next(); + merger.reset(); + merger.add(candidate); + } + else + { + candidates.add(range.next()); + } + + processedRanges.add(range); + } + + ranges.addAll(processedRanges); + return merger.merge(); + } + + protected void performSkipTo(Long nextToken) + { + while (!candidates.isEmpty()) + { + Token candidate = candidates.peek(); + if (candidate.get() >= nextToken) + break; + candidates.poll(); + } + while (!ranges.isEmpty()) + { + if (ranges.peek().getCurrent().compareTo(nextToken) >= 0) + break; + + RangeIterator head = ranges.poll(); + + if (head.getMaximum().compareTo(nextToken) >= 0) + { + head.skipTo(nextToken); + if (head.hasNext()) + { + ranges.add(head); + continue; + } + } + } + } + + public void close() throws IOException + { + // Due to lazy key fetching, we cannot close iterator immediately + toRelease.forEach(FileUtils::closeQuietly); + ranges.forEach(FileUtils::closeQuietly); + } + + public static Builder builder() + { + return new Builder(); + } + + public static RangeIterator build(List tokens) + { + return new Builder().add(tokens).build(); + } + + public static class Builder extends RangeIterator.Builder + { + public Builder() + { + super(IteratorType.UNION); + } + + protected RangeIterator buildIterator() + { + switch (rangeCount()) + { + case 1: + return ranges.poll(); + + default: + return new RangeUnionIterator(statistics, ranges); + } + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/SAICodecUtils.java b/src/java/org/apache/cassandra/index/sai/utils/SAICodecUtils.java new file mode 100644 index 000000000000..a588f4cc59c6 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/SAICodecUtils.java @@ -0,0 +1,266 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.utils; + +import java.io.IOException; + +import org.apache.cassandra.index.sai.disk.format.Version; +import org.apache.cassandra.io.compress.CorruptBlockException; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; + +import static org.apache.lucene.codecs.CodecUtil.CODEC_MAGIC; +import static org.apache.lucene.codecs.CodecUtil.FOOTER_MAGIC; + +public class SAICodecUtils +{ + public static final String FOOTER_POINTER = "footerPointer"; + + public static void writeHeader(IndexOutput out) throws IOException + { + out.writeInt(CODEC_MAGIC); + out.writeString(Version.LATEST.toString()); + } + + public static void writeFooter(IndexOutput out) throws IOException + { + out.writeInt(FOOTER_MAGIC); + out.writeInt(0); + writeCRC(out); + } + + public static Version checkHeader(DataInput in) throws IOException + { + try + { + final int actualMagic = in.readInt(); + if (actualMagic != CODEC_MAGIC) + { + throw new CorruptIndexException("codec header mismatch: actual header=" + actualMagic + " vs expected header=" + CODEC_MAGIC, in); + } + final Version actualVersion = Version.parse(in.readString()); + if (!actualVersion.onOrAfter(Version.EARLIEST)) + { + throw new IOException("Unsupported version: " + actualVersion); + } + return actualVersion; + } + catch (Throwable th) + { + if (th.getCause() instanceof CorruptBlockException) + { + throw new CorruptIndexException("corrupted", in, th.getCause()); + } + else + { + throw th; + } + } + } + + public static long checkFooter(ChecksumIndexInput in) throws IOException + { + validateFooter(in, false); + long actualChecksum = in.getChecksum(); + long expectedChecksum = readCRC(in); + if (expectedChecksum != actualChecksum) + { + throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + Long.toHexString(expectedChecksum) + + " actual=" + Long.toHexString(actualChecksum), in); + } + return actualChecksum; + } + + public static void validate(IndexInput input) throws IOException + { + checkHeader(input); + validateFooterAndResetPosition(input); + } + + public static void validate(IndexInput input, long footerPointer) throws IOException + { + checkHeader(input); + + long current = input.getFilePointer(); + input.seek(footerPointer); + validateFooter(input, true); + + input.seek(current); + } + + public static void validateFooterAndResetPosition(IndexInput in) throws IOException + { + long position = in.getFilePointer(); + long fileLength = in.length(); + long footerLength = CodecUtil.footerLength(); + long footerPosition = fileLength - footerLength; + + if (footerPosition < 0) + { + throw new CorruptIndexException("invalid codec footer (file truncated?): file length=" + fileLength + ", footer length=" + footerLength, in); + } + + in.seek(footerPosition); + validateFooter(in, false); + in.seek(position); + } + + public static void validateChecksum(IndexInput input) throws IOException + { + long position = input.getFilePointer(); + long expected = CodecUtil.retrieveChecksum(input); + + input.seek(position); + long actual = CodecUtil.checksumEntireFile(input); + if (expected != actual) + throw new CorruptIndexException("checksum failed (hardware problem?) : expected=" + Long.toHexString(expected) + " actual=" + Long.toHexString(actual), input); + } + + /** + * Copied from org.apache.lucene.codecs.CodecUtil.validateFooter(IndexInput) + */ + public static void validateFooter(IndexInput in, boolean padded) throws IOException + { + long remaining = in.length() - in.getFilePointer(); + long expected = CodecUtil.footerLength(); + + if (!padded) + { + if (remaining < expected) + { + throw new CorruptIndexException("misplaced codec footer (file truncated?): remaining=" + remaining + ", expected=" + expected + ", fp=" + in.getFilePointer(), in); + } + else if (remaining > expected) + { + throw new CorruptIndexException("misplaced codec footer (file extended?): remaining=" + remaining + ", expected=" + expected + ", fp=" + in.getFilePointer(), in); + } + } + + final int magic = in.readInt(); + + if (magic != FOOTER_MAGIC) + { + throw new CorruptIndexException("codec footer mismatch (file truncated?): actual footer=" + magic + " vs expected footer=" + FOOTER_MAGIC, in); + } + + final int algorithmID = in.readInt(); + + if (algorithmID != 0) + { + throw new CorruptIndexException("codec footer mismatch: unknown algorithmID: " + algorithmID, in); + } + } + + + // Copied from Lucene CodecUtil as they are not public + + /** + * Writes CRC32 value as a 64-bit long to the output. + * @throws IllegalStateException if CRC is formatted incorrectly (wrong bits set) + * @throws IOException if an i/o error occurs + */ + static void writeCRC(IndexOutput output) throws IOException { + long value = output.getChecksum(); + if ((value & 0xFFFFFFFF00000000L) != 0) { + throw new IllegalStateException("Illegal CRC-32 checksum: " + value + " (resource=" + output + ")"); + } + output.writeLong(value); + } + + /** + * Reads CRC32 value as a 64-bit long from the input. + * @throws CorruptIndexException if CRC is formatted incorrectly (wrong bits set) + * @throws IOException if an i/o error occurs + */ + static long readCRC(IndexInput input) throws IOException { + long value = input.readLong(); + if ((value & 0xFFFFFFFF00000000L) != 0) { + throw new CorruptIndexException("Illegal CRC-32 checksum: " + value, input); + } + return value; + } + + // Copied from Lucene PackedInts as they are not public + + public static int checkBlockSize(int blockSize, int minBlockSize, int maxBlockSize) { + if (blockSize >= minBlockSize && blockSize <= maxBlockSize) { + if ((blockSize & blockSize - 1) != 0) { + throw new IllegalArgumentException("blockSize must be a power of two, got " + blockSize); + } else { + return Integer.numberOfTrailingZeros(blockSize); + } + } else { + throw new IllegalArgumentException("blockSize must be >= " + minBlockSize + " and <= " + maxBlockSize + ", got " + blockSize); + } + } + + public static int numBlocks(long size, int blockSize) { + int numBlocks = (int)(size / (long)blockSize) + (size % (long)blockSize == 0L ? 0 : 1); + if ((long)numBlocks * (long)blockSize < size) { + throw new IllegalArgumentException("size is too large for this block size"); + } else { + return numBlocks; + } + } + + // Copied from Lucene BlockPackedReaderIterator as they are not public + + /** + * Same as DataInput.readVLong but supports negative values + */ + public static long readVLong(DataInput in) throws IOException + { + byte b = in.readByte(); + if (b >= 0) return b; + long i = b & 0x7FL; + b = in.readByte(); + i |= (b & 0x7FL) << 7; + if (b >= 0) return i; + b = in.readByte(); + i |= (b & 0x7FL) << 14; + if (b >= 0) return i; + b = in.readByte(); + i |= (b & 0x7FL) << 21; + if (b >= 0) return i; + b = in.readByte(); + i |= (b & 0x7FL) << 28; + if (b >= 0) return i; + b = in.readByte(); + i |= (b & 0x7FL) << 35; + if (b >= 0) return i; + b = in.readByte(); + i |= (b & 0x7FL) << 42; + if (b >= 0) return i; + b = in.readByte(); + i |= (b & 0x7FL) << 49; + if (b >= 0) return i; + b = in.readByte(); + i |= (b & 0xFFL) << 56; + return i; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/SeekingRandomAccessInput.java b/src/java/org/apache/cassandra/index/sai/utils/SeekingRandomAccessInput.java new file mode 100644 index 000000000000..64d66daebfdc --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/SeekingRandomAccessInput.java @@ -0,0 +1,70 @@ +/* + * 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.index.sai.utils; + +import java.io.IOException; + +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; + +/** + * {@link IndexInput} adapter that exposes it as a {@link RandomAccessInput} type. + */ +public class SeekingRandomAccessInput implements RandomAccessInput +{ + private final IndexInput in; + + public SeekingRandomAccessInput(IndexInput in) + { + this.in = in; + } + + @Override + public byte readByte(long pos) throws IOException + { + in.seek(pos); + return in.readByte(); + } + + @Override + public short readShort(long pos) throws IOException + { + in.seek(pos); + return in.readShort(); + } + + @Override + public int readInt(long pos) throws IOException + { + in.seek(pos); + return in.readInt(); + } + + @Override + public long readLong(long pos) throws IOException + { + in.seek(pos); + return in.readLong(); + } + + @Override + public String toString() + { + return "SeekingRandomAccessInput(" + in + ")"; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/TermIterator.java b/src/java/org/apache/cassandra/index/sai/utils/TermIterator.java new file mode 100644 index 000000000000..c58ad339571f --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/TermIterator.java @@ -0,0 +1,135 @@ +/* + * 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.index.sai.utils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.index.sai.QueryContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.SSTableQueryContext; +import org.apache.cassandra.index.sai.Token; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Throwables; + +public class TermIterator extends RangeIterator +{ + private static final Logger logger = LoggerFactory.getLogger(TermIterator.class); + + private final QueryContext context; + + private final RangeIterator union; + private final Set referencedIndexes; + + private TermIterator(RangeIterator union, Set referencedIndexes, QueryContext queryContext) + { + super(union.getMinimum(), union.getMaximum(), union.getCount()); + + this.union = union; + this.referencedIndexes = referencedIndexes; + this.context = queryContext; + } + + @SuppressWarnings("resource") + public static TermIterator build(final Expression e, Set perSSTableIndexes, AbstractBounds keyRange, QueryContext queryContext, boolean defer) + { + final List tokens = new ArrayList<>(1 + perSSTableIndexes.size());; + + RangeIterator memtableIterator = e.context.searchMemtable(e, keyRange); + if (memtableIterator != null) + tokens.add(memtableIterator); + + for (final SSTableIndex index : perSSTableIndexes) + { + try + { + queryContext.checkpoint(); + queryContext.incSstablesHit(); + assert !index.isReleased(); + + SSTableQueryContext context = queryContext.getSSTableQueryContext(index.getSSTable()); + RangeIterator keyIterator = index.search(e, keyRange, context, defer); + + if (keyIterator == null) + continue; + + tokens.add(keyIterator); + } + catch (Throwable e1) + { + if (logger.isDebugEnabled() && !(e1 instanceof AbortedOperationException)) + logger.debug(String.format("Failed search an index %s, skipping.", index.getSSTable()), e1); + + throw Throwables.cleaned(e1); + } + } + + RangeIterator ranges = RangeUnionIterator.build(tokens); + return new TermIterator(ranges, perSSTableIndexes, queryContext); + } + + protected Token computeNext() + { + try + { + return union.hasNext() ? union.next() : endOfData(); + } + finally + { + context.checkpoint(); + } + } + + protected void performSkipTo(Long nextToken) + { + try + { + union.skipTo(nextToken); + } + finally + { + context.checkpoint(); + } + } + + public void close() + { + FileUtils.closeQuietly(union); + referencedIndexes.forEach(TermIterator::releaseQuietly); + referencedIndexes.clear(); + } + + private static void releaseQuietly(SSTableIndex index) + { + try + { + index.release(); + } + catch (Throwable e) + { + logger.error(String.format("Failed to release index %s", index.getSSTable()), e); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java b/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java new file mode 100644 index 000000000000..b6c0471affc2 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java @@ -0,0 +1,543 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.utils; + +import java.math.BigInteger; +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Set; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import com.googlecode.concurrenttrees.radix.ConcurrentRadixTree; +import org.apache.cassandra.cql3.statements.schema.IndexTarget; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.DecimalType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.ComplexColumnData; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FastByteOperations; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; + +public class TypeUtil +{ + private static final byte[] IPV4_PREFIX = new byte[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -1, -1 }; + + /** + * DecimalType / BigDecimal values are indexed by truncating their asComparableBytes representation to this size, + * padding on the right with zero-value-bytes until this size is reached (if necessary). This causes + * false-positives that must be filtered in a separate step after hitting the index and reading the associated + * (full) values. + */ + public static final int DECIMAL_APPROXIMATION_BYTES = 24; + + private TypeUtil() {} + + /** + * Returns true if given buffer would pass the {@link AbstractType#validate(ByteBuffer)} + * check. False otherwise. + */ + public static boolean isValid(ByteBuffer term, AbstractType validator) + { + try + { + validator.validate(term); + return true; + } + catch (MarshalException e) + { + return false; + } + } + + /** + * Indicates if the type encoding supports rounding of the raw value. + * + * This is significant in range searches where we have to make all range + * queries inclusive when searching the indexes in order to avoid excluding + * rounded values. Excluded values are removed by post-filtering. + */ + public static boolean supportsRounding(AbstractType type) + { + return isBigInteger(type) || isBigDecimal(type); + } + + /** + * Returns the smaller of two {@code ByteBuffer} values, based on the result of {@link + * #compare(ByteBuffer, ByteBuffer, AbstractType)} comparision. + */ + public static ByteBuffer min(ByteBuffer a, ByteBuffer b, AbstractType type) + { + return a == null ? b : (b == null || compare(b, a, type) > 0) ? a : b; + } + + /** + * Returns the greater of two {@code ByteBuffer} values, based on the result of {@link + * #compare(ByteBuffer, ByteBuffer, AbstractType)} comparision. + */ + public static ByteBuffer max(ByteBuffer a, ByteBuffer b, AbstractType type) + { + return a == null ? b : (b == null || compare(b, a, type) < 0) ? a : b; + } + + /** + * Returns the lesser of two {@code ByteComparable} values, based on the result of {@link + * ByteComparable#compare(ByteComparable, ByteComparable, ByteComparable.Version)} comparision. + */ + public static ByteComparable min(ByteComparable a, ByteComparable b) + { + return a == null ? b : (b == null || ByteComparable.compare(b, a, ByteComparable.Version.OSS41) > 0) ? a : b; + } + + /** + * Returns the greater of two {@code ByteComparable} values, based on the result of {@link + * ByteComparable#compare(ByteComparable, ByteComparable, ByteComparable.Version)} comparision. + */ + public static ByteComparable max(ByteComparable a, ByteComparable b) + { + return a == null ? b : (b == null || ByteComparable.compare(b, a, ByteComparable.Version.OSS41) < 0) ? a : b; + } + + /** + * Returns the value length for the given {@link AbstractType}, selecting 16 for types + * that officially use VARIABLE_LENGTH but are, in fact, of a fixed length. + */ + public static int fixedSizeOf(AbstractType type) + { + if (type.isValueLengthFixed()) + return type.valueLengthIfFixed(); + else if (isInetAddress(type)) + return 16; + else if (isBigInteger(type)) + return 20; + else if (type instanceof DecimalType) + return DECIMAL_APPROXIMATION_BYTES; + return 16; + } + + public static AbstractType cellValueType(Pair target) + { + AbstractType type = target.left.type; + if (isNonFrozenCollection(type)) + { + CollectionType collection = ((CollectionType) type); + switch (collection.kind) + { + case LIST: + return collection.valueComparator(); + case SET: + return collection.nameComparator(); + case MAP: + switch (target.right) + { + case KEYS: + return collection.nameComparator(); + case VALUES: + return collection.valueComparator(); + case KEYS_AND_VALUES: + return CompositeType.getInstance(collection.nameComparator(), collection.valueComparator()); + } + } + } + return type; + } + + /** + * Allows overriding the default getString method for {@link CompositeType}. It is + * a requirement of the {@link ConcurrentRadixTree} that the keys are strings but + * the getString method of {@link CompositeType} does not return a string that compares + * in the same order as the underlying {@link ByteBuffer}. To get round this we convert + * the {@link CompositeType} bytes to a hex string. + */ + public static String getString(ByteBuffer value, AbstractType type) + { + if (isComposite(type)) + return ByteBufferUtil.bytesToHex(value); + return type.getString(value); + } + + /** + * The inverse of the above method. Overrides the fromString method on {@link CompositeType} + * in order to convert the hex string to bytes. + */ + public static ByteBuffer fromString(String value, AbstractType type) + { + if (isComposite(type)) + return ByteBufferUtil.hexToBytes(value); + return type.fromString(value); + } + + public static ByteSource asComparableBytes(ByteBuffer value, AbstractType type, ByteComparable.Version version) + { + if (type instanceof InetAddressType || type instanceof IntegerType || type instanceof DecimalType) + return ByteSource.optionalFixedLength(ByteBufferAccessor.instance, value); + return type.asComparableBytes(value, version); + } + + /** + * Fills a byte array with the comparable bytes for a type. + *

    + * This method expects a {@code value} parameter generated by calling {@link #encode(ByteBuffer, AbstractType)}. + * It is not generally safe to pass the output of other serialization methods to this method. For instance, it is + * not generally safe to pass the output of {@link AbstractType#decompose(Object)} as the {@code value} parameter + * (there are certain types for which this is technically OK, but that doesn't hold for all types). + * + * @param value a value buffer returned by {@link #encode(ByteBuffer, AbstractType)} + * @param type the type associated with the encoded {@code value} parameter + * @param bytes this method's output + */ + public static void toComparableBytes(ByteBuffer value, AbstractType type, byte[] bytes) + { + if (isInetAddress(type)) + ByteBufferUtil.arrayCopy(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, 16); + else if (isBigInteger(type)) + ByteBufferUtil.arrayCopy(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, 20); + else if (type instanceof DecimalType) + ByteBufferUtil.arrayCopy(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, DECIMAL_APPROXIMATION_BYTES); + else + ByteBufferUtil.toBytes(type.asComparableBytes(value, ByteComparable.Version.OSS41), bytes); + } + + /** + * Encode an external term from a memtable index or a compaction. The purpose of this is to + * allow terms of particular types to be handled differently and not use the default + * {@link ByteComparable} encoding. + */ + public static ByteBuffer encode(ByteBuffer value, AbstractType type) + { + if (value == null) + return null; + + if (isInetAddress(type)) + return encodeInetAddress(value); + else if (isBigInteger(type)) + return encodeBigInteger(value); + else if (type instanceof DecimalType) + return encodeDecimal(value); + return value; + } + + /** + * Compare two terms based on their type. This is used in place of {@link AbstractType#compare(ByteBuffer, ByteBuffer)} + * so that the default comparison can be overridden for specific types. + * + * Note: This should be used for all term comparison + */ + public static int compare(ByteBuffer b1, ByteBuffer b2, AbstractType type) + { + if (isInetAddress(type)) + return compareInet(b1, b2); + // BigInteger values, frozen types and composite types (map entries) use compareUnsigned to maintain + // a consistent order between the in-memory index and the on-disk index. + else if (isBigInteger(type) || isBigDecimal(type) || isCompositeOrFrozen(type)) + return FastByteOperations.compareUnsigned(b1, b2); + + return type.compare(b1, b2 ); + } + + /** + * This is used for value comparison in post-filtering - {@link Expression#isSatisfiedBy(ByteBuffer)}. + * + * This allows types to decide whether they should be compared based on their encoded value or their + * raw value. At present only {@link InetAddressType} values are compared by their encoded values to + * allow for ipv4 -> ipv6 equivalency in searches. + */ + public static int comparePostFilter(Expression.Value requestedValue, Expression.Value columnValue, AbstractType type) + { + if (isInetAddress(type)) + return compareInet(requestedValue.encoded, columnValue.encoded); + // Override comparisons for frozen collections and composite types (map entries) + else if (isCompositeOrFrozen(type)) + return FastByteOperations.compareUnsigned(requestedValue.raw, columnValue.raw); + + return type.compare(requestedValue.raw, columnValue.raw); + } + + public static Iterator collectionIterator(AbstractType validator, + ComplexColumnData cellData, + Pair target, + int nowInSecs) + { + if (cellData == null) + return null; + + Stream stream = StreamSupport.stream(cellData.spliterator(), false).filter(cell -> cell != null && cell.isLive(nowInSecs)) + .map(cell -> cellValue(target, cell)); + + if (isInetAddress(validator)) + stream = stream.sorted((c1, c2) -> compareInet(encodeInetAddress(c1), encodeInetAddress(c2))); + + return stream.iterator(); + } + + public static Comparator comparator(AbstractType type) + { + // Override the comparator for BigInteger, frozen collections and composite types + if (isBigInteger(type) || isBigDecimal(type) || isCompositeOrFrozen(type)) + return FastByteOperations::compareUnsigned; + + return type; + } + + private static ByteBuffer cellValue(Pair target, Cell cell) + { + if (target.left.type.isCollection() && target.left.type.isMultiCell()) + { + switch (((CollectionType) target.left.type).kind) + { + case LIST: + //TODO Is there any optimisation can be done here with cell values? + return cell.buffer(); + case SET: + return cell.path().get(0); + case MAP: + switch (target.right) + { + case KEYS: + return cell.path().get(0); + case VALUES: + return cell.buffer(); + case KEYS_AND_VALUES: + return CompositeType.build(ByteBufferAccessor.instance, cell.path().get(0), cell.buffer()); + } + } + } + return cell.buffer(); + } + + /** + * Compares 2 InetAddress terms by ensuring that both addresses are represented as + * ipv6 addresses. + */ + private static int compareInet(ByteBuffer b1, ByteBuffer b2) + { + assert isIPv6(b1) && isIPv6(b2); + + return FastByteOperations.compareUnsigned(b1, b2); + } + + private static boolean isIPv6(ByteBuffer address) + { + return address.remaining() == 16; + } + + /** + * Encode a {@link InetAddress} into a fixed width 16 byte encoded value. + * + * The encoded value is byte comparable and prefix compressible. + * + * The encoding is done by converting ipv4 addresses to their ipv6 equivalent. + */ + private static ByteBuffer encodeInetAddress(ByteBuffer value) + { + if (value.remaining() == 4) + { + int position = value.hasArray() ? value.arrayOffset() + value.position() : value.position(); + ByteBuffer mapped = ByteBuffer.allocate(16); + System.arraycopy(IPV4_PREFIX, 0, mapped.array(), 0, IPV4_PREFIX.length); + ByteBufferUtil.arrayCopy(value, position, mapped, IPV4_PREFIX.length, value.remaining()); + return mapped; + } + return value; + } + + /** + * Encode a {@link BigInteger} into a fixed width 20 byte encoded value. + * + * The encoded value is byte comparable and prefix compressible. + * + * The format of the encoding is: + * + * The first 4 bytes contain the length of the {@link BigInteger} byte array + * with the top bit flipped for positive values. + * + * The remaining 16 bytes contain the 16 most significant bytes of the + * {@link BigInteger} byte array. + * + * For {@link BigInteger} values whose underlying byte array is less than + * 16 bytes, the encoded value is sign extended. + */ + public static ByteBuffer encodeBigInteger(ByteBuffer value) + { + int size = value.remaining(); + int position = value.hasArray() ? value.arrayOffset() + value.position() : value.position(); + byte[] bytes = new byte[20]; + if (size < 16) + { + ByteBufferUtil.arrayCopy(value, position, bytes, bytes.length - size, size); + if ((bytes[bytes.length - size] & 0x80) != 0) + Arrays.fill(bytes, 4, bytes.length - size, (byte)0xff); + else + Arrays.fill(bytes, 4, bytes.length - size, (byte)0x00); + } + else + { + ByteBufferUtil.arrayCopy(value, position, bytes, 4, 16); + } + if ((bytes[4] & 0x80) != 0) + { + size = -size; + } + bytes[0] = (byte)(size >> 24 & 0xff); + bytes[1] = (byte)(size >> 16 & 0xff); + bytes[2] = (byte)(size >> 8 & 0xff); + bytes[3] = (byte)(size & 0xff); + bytes[0] ^= 0x80; + return ByteBuffer.wrap(bytes); + } + + /* Type comparison to get rid of ReversedType */ + + /** + * Returns true if values of the given {@link AbstractType} should be indexed as literals. + */ + public static boolean isLiteral(AbstractType type) + { + return isUTF8OrAscii(type) || isCompositeOrFrozen(type) || baseType(type) instanceof BooleanType; + } + + /** + * Returns true if given {@link AbstractType} is UTF8 or Ascii + */ + public static boolean isUTF8OrAscii(AbstractType type) + { + type = baseType(type); + return type instanceof UTF8Type || type instanceof AsciiType; + } + + /** + * Returns true if given {@link AbstractType} is a Composite(map entry) or frozen. + */ + public static boolean isCompositeOrFrozen(AbstractType type) + { + type = baseType(type); + return type instanceof CompositeType || isFrozen(type); + } + + /** + * Returns true if given {@link AbstractType} is frozen. + */ + public static boolean isFrozen(AbstractType type) + { + type = baseType(type); + return !type.subTypes().isEmpty() && !type.isMultiCell(); + } + + /** + * Returns true if given {@link AbstractType} is a frozen collection. + */ + public static boolean isFrozenCollection(AbstractType type) + { + type = baseType(type); + return type.isCollection() && !type.isMultiCell(); + } + + /** + * Returns true if given {@link AbstractType} is a non-frozen collection. + */ + public static boolean isNonFrozenCollection(AbstractType type) + { + type = baseType(type); + return type.isCollection() && type.isMultiCell(); + } + + /** + * Returns true if given {@link AbstractType} is included in the types. + */ + public static boolean isIn(AbstractType type, Set> types) + { + type = baseType(type); + return types.contains(type); + } + + /** + * Returns true if given {@link AbstractType} is {@link InetAddressType} + */ + private static boolean isInetAddress(AbstractType type) + { + type = baseType(type); + return type instanceof InetAddressType; + } + + /** + * Returns true if given {@link AbstractType} is {@link IntegerType} + */ + private static boolean isBigInteger(AbstractType type) + { + type = baseType(type); + return type instanceof IntegerType; + } + + /** + * Returns true if given {@link AbstractType} is {@link DecimalType} + */ + private static boolean isBigDecimal(AbstractType type) + { + type = baseType(type); + return type instanceof DecimalType; + } + + /** + * Returns true if given {@link AbstractType} is {@link CompositeType} + */ + public static boolean isComposite(AbstractType type) + { + type = baseType(type); + return type instanceof CompositeType; + } + + /** + * @return base type if given type is reversed, otherwise return itself + */ + private static AbstractType baseType(AbstractType type) + { + return type.isReversed() ? ((ReversedType) type).baseType : type; + } + + public static ByteBuffer encodeDecimal(ByteBuffer value) + { + ByteSource bs = DecimalType.instance.asComparableBytes(value, ByteComparable.Version.OSS41); + bs = ByteSource.cutOrRightPad(bs, DECIMAL_APPROXIMATION_BYTES, 0); + return ByteBuffer.wrap(ByteSourceInverse.readBytes(bs, DECIMAL_APPROXIMATION_BYTES)); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/view/IndexViewManager.java b/src/java/org/apache/cassandra/index/sai/view/IndexViewManager.java new file mode 100644 index 000000000000..d69227c37770 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/view/IndexViewManager.java @@ -0,0 +1,161 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.view; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.StorageAttachedIndexGroup; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Pair; + +/** + * Maintain a atomic view for read requests, so that requests can read all data during concurrent compactions. + * + * All per-column {@link SSTableIndex} updates should be proxied by {@link StorageAttachedIndexGroup} to make + * sure per-sstable {@link SSTableContext} are in-sync. + */ +public class IndexViewManager +{ + private static final Logger logger = LoggerFactory.getLogger(IndexViewManager.class); + + private final ColumnContext context; + private final AtomicReference view = new AtomicReference<>(); + + public IndexViewManager(ColumnContext context) + { + this(context, Collections.emptySet()); + } + + @VisibleForTesting + IndexViewManager(ColumnContext context, Collection indices) + { + this.context = context; + this.view.set(new View(context, indices)); + } + + public View getView() + { + return view.get(); + } + + /** + * Replaces old SSTables with new by creating new immutable view. + * + * @param oldSSTables A set of SSTables to remove. + * @param newSSTableContexts A set of SSTableContexts to add to tracker. + * @param validate if true, per-column index files' header and footer will be validated. + * @param rename if true check whether the per-column index components need renaming + * + * @return A set of SSTables which have attached to them invalid index components. + */ + public Set update(Collection oldSSTables, Collection newSSTableContexts, boolean validate, boolean rename) + { + // Valid indexes on the left and invalid SSTable contexts on the right... + Pair, Set> indexes = context.getBuiltIndexes(newSSTableContexts, validate, rename); + + View currentView, newView; + Collection newViewIndexes = new HashSet<>(); + Collection releasableIndexes = new ArrayList<>(); + Collection toRemove = new HashSet<>(oldSSTables); + + do + { + currentView = view.get(); + newViewIndexes.clear(); + releasableIndexes.clear(); + + for (SSTableIndex sstableIndex : currentView) + { + // When aborting early open transaction, toRemove may have the same sstable files as newSSTableContexts, + // but different SSTableReader java objects with different start positions. So we need to release them + // from existing view. see DSP-19677 + SSTableReader sstable = sstableIndex.getSSTable(); + if (toRemove.contains(sstable) || newViewIndexes.contains(sstableIndex)) + releasableIndexes.add(sstableIndex); + else + newViewIndexes.add(sstableIndex); + } + + for (SSTableIndex sstableIndex : indexes.left) + { + if (newViewIndexes.contains(sstableIndex)) + releasableIndexes.add(sstableIndex); + else + newViewIndexes.add(sstableIndex); + } + + newView = new View(context, newViewIndexes); + } + while (!view.compareAndSet(currentView, newView)); + + releasableIndexes.forEach(SSTableIndex::release); + + if (logger.isTraceEnabled()) + logger.trace(context.logMessage("There are now {} active SSTable indexes."), view.get().getIndexes().size()); + + return indexes.right; + } + + public void drop(Collection sstablesToRebuild) + { + View currentView = view.get(); + + Set toRemove = new HashSet<>(sstablesToRebuild); + for (SSTableIndex index : currentView) + { + SSTableReader sstable = index.getSSTable(); + if (!toRemove.contains(sstable)) + continue; + + index.markObsolete(); + } + + update(toRemove, Collections.emptyList(), false, false); + } + + public void invalidate() + { + View currentView = view.get(); + + for (SSTableIndex index : currentView) + { + index.markObsolete(); + } + + view.set(new View(context, Collections.emptyList())); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/view/RangeTermTree.java b/src/java/org/apache/cassandra/index/sai/view/RangeTermTree.java new file mode 100644 index 000000000000..d2f6375b0ceb --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/view/RangeTermTree.java @@ -0,0 +1,124 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.view; + +import java.lang.invoke.MethodHandles; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.utils.Interval; +import org.apache.cassandra.utils.IntervalTree; + +public class RangeTermTree implements TermTree +{ + private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + protected final ByteBuffer min, max; + protected final AbstractType comparator; + + private final IntervalTree> rangeTree; + + private RangeTermTree(ByteBuffer min, ByteBuffer max, IntervalTree> rangeTree, AbstractType comparator) + { + this.min = min; + this.max = max; + this.rangeTree = rangeTree; + this.comparator = comparator; + } + + public Set search(Expression e) + { + ByteBuffer minTerm = e.lower == null ? min : e.lower.value.encoded; + ByteBuffer maxTerm = e.upper == null ? max : e.upper.value.encoded; + + return new HashSet<>(rangeTree.search(Interval.create(new Term(minTerm, comparator), + new Term(maxTerm, comparator), + null))); + } + + static class Builder extends TermTree.Builder + { + final List> intervals = new ArrayList<>(); + + protected Builder(AbstractType comparator) + { + super(comparator); + } + + public void addIndex(SSTableIndex index) + { + Interval interval = + Interval.create(new Term(index.minTerm(), comparator), new Term(index.maxTerm(), comparator), index); + + if (logger.isTraceEnabled()) + { + ColumnContext context = index.getColumnContext(); + logger.trace(context.logMessage("Adding index for SSTable {} with minTerm={} and maxTerm={}..."), + index.getSSTable().descriptor, + comparator.compose(index.minTerm()), + comparator.compose(index.maxTerm())); + } + + intervals.add(interval); + } + + public TermTree build() + { + return new RangeTermTree(min, max, IntervalTree.build(intervals), comparator); + } + } + + /** + * This is required since IntervalTree doesn't support custom Comparator + * implementations and relied on items to be comparable which "raw" terms are not. + */ + protected static class Term implements Comparable + { + private final ByteBuffer term; + private final AbstractType comparator; + + Term(ByteBuffer term, AbstractType comparator) + { + this.term = term; + this.comparator = comparator; + } + + public int compareTo(Term o) + { + return TypeUtil.compare(term, o.term, comparator); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/view/TermTree.java b/src/java/org/apache/cassandra/index/sai/view/TermTree.java new file mode 100644 index 000000000000..d2041b29f83c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/view/TermTree.java @@ -0,0 +1,61 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.view; + +import java.nio.ByteBuffer; +import java.util.Set; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.index.sai.utils.TypeUtil; + +public interface TermTree +{ + Set search(Expression e); + + abstract class Builder + { + protected final AbstractType comparator; + protected ByteBuffer min, max; + + protected Builder(AbstractType comparator) + { + this.comparator = comparator; + } + + public final void add(SSTableIndex index) + { + addIndex(index); + + min = min == null || TypeUtil.compare(min, index.minTerm(), comparator) > 0 ? index.minTerm() : min; + max = max == null || TypeUtil.compare(max, index.maxTerm(), comparator) < 0 ? index.maxTerm() : max; + } + + protected abstract void addIndex(SSTableIndex index); + + public abstract TermTree build(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/view/View.java b/src/java/org/apache/cassandra/index/sai/view/View.java new file mode 100644 index 000000000000..d10ac1589c6d --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/view/View.java @@ -0,0 +1,129 @@ +/* + * All changes to the original code are Copyright DataStax, Inc. + * + * Please see the included license file for details. + */ + +/* + * 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.index.sai.view; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.plan.Expression; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.Interval; +import org.apache.cassandra.utils.IntervalTree; + +public class View implements Iterable +{ + private final Map view; + + private final TermTree termTree; + private final AbstractType keyValidator; + private final IntervalTree> keyIntervalTree; + + public View(ColumnContext context, Collection indexes) { + this.view = new HashMap<>(); + this.keyValidator = context.keyValidator(); + + AbstractType validator = context.getValidator(); + + TermTree.Builder termTreeBuilder = new RangeTermTree.Builder(validator); + + List> keyIntervals = new ArrayList<>(); + for (SSTableIndex sstableIndex : indexes) + { + this.view.put(sstableIndex.getSSTable().descriptor, sstableIndex); + termTreeBuilder.add(sstableIndex); + keyIntervals.add(Interval.create(new Key(sstableIndex.minKey()), + new Key(sstableIndex.maxKey()), + sstableIndex)); + } + + this.termTree = termTreeBuilder.build(); + this.keyIntervalTree = IntervalTree.build(keyIntervals); + } + + public Set match(Expression expression) + { + return termTree.search(expression); + } + + public List match(DecoratedKey minKey, DecoratedKey maxKey) + { + return keyIntervalTree.search(Interval.create(new Key(minKey), new Key(maxKey), null)); + } + + public Iterator iterator() + { + return view.values().iterator(); + } + + public Collection getIndexes() + { + return view.values(); + } + + public boolean containsSSTable(SSTableReader sstable) + { + return view.containsKey(sstable.descriptor); + } + + public int size() + { + return view.size(); + } + + /** + * This is required since IntervalTree doesn't support custom Comparator + * implementations and relied on items to be comparable which "raw" keys are not. + */ + private static class Key implements Comparable + { + private final DecoratedKey key; + + public Key(DecoratedKey key) + { + this.key = key; + } + + public int compareTo(Key o) + { + return key.compareTo(o.key); + } + } + + @Override + public String toString() + { + return String.format("View{view=%s, keyValidator=%s, keyIntervalTree=%s}", view, keyValidator, keyIntervalTree); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/virtual/IndexesSystemView.java b/src/java/org/apache/cassandra/index/sai/virtual/IndexesSystemView.java new file mode 100644 index 000000000000..ba5e0f717815 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/virtual/IndexesSystemView.java @@ -0,0 +1,153 @@ +/* + * 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.index.sai.virtual; + +import java.util.function.Consumer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.virtual.AbstractVirtualTable; +import org.apache.cassandra.db.virtual.SimpleDataSet; +import org.apache.cassandra.db.virtual.VirtualTable; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.SecondaryIndexManager; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.index.sai.StorageAttachedIndexGroup; +import org.apache.cassandra.index.sai.view.View; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadata; + +/** + * A {@link VirtualTable} providing a system view of per-column storage-attached index metadata. + */ +public class IndexesSystemView extends AbstractVirtualTable +{ + private static final Logger logger = LoggerFactory.getLogger(IndexesSystemView.class); + + static final String NAME = "indexes"; + + static final String KEYSPACE_NAME = "keyspace_name"; + static final String INDEX_NAME = "index_name"; + static final String TABLE_NAME = "table_name"; + static final String COLUMN_NAME = "column_name"; + static final String IS_QUERYABLE = "is_queryable"; + static final String IS_BUILDING = "is_building"; + static final String IS_STRING = "is_string"; + static final String ANALYZER = "analyzer"; + static final String INDEXED_SSTABLE_COUNT = "indexed_sstable_count"; + static final String CELL_COUNT = "cell_count"; + static final String PER_TABLE_DISK_SIZE = "per_table_disk_size"; + static final String PER_COLUMN_DISK_SIZE = "per_column_disk_size"; + static final String PER_TABLE_FILE_CACHE_SIZE = "per_table_file_cache_size"; + static final String PER_COLUMN_FILE_CACHE_SIZE = "per_column_file_cache_size"; + + public IndexesSystemView(String keyspace) + { + super(TableMetadata.builder(keyspace, NAME) + .partitioner(new LocalPartitioner(UTF8Type.instance)) + .comment("Storage-attached column index metadata") + .kind(TableMetadata.Kind.VIRTUAL) + .addPartitionKeyColumn(KEYSPACE_NAME, UTF8Type.instance) + .addClusteringColumn(INDEX_NAME, UTF8Type.instance) + .addRegularColumn(TABLE_NAME, UTF8Type.instance) + .addRegularColumn(COLUMN_NAME, UTF8Type.instance) + .addRegularColumn(IS_QUERYABLE, BooleanType.instance) + .addRegularColumn(IS_BUILDING, BooleanType.instance) + .addRegularColumn(IS_STRING, BooleanType.instance) + .addRegularColumn(ANALYZER, UTF8Type.instance) + .addRegularColumn(INDEXED_SSTABLE_COUNT, Int32Type.instance) + .addRegularColumn(CELL_COUNT, LongType.instance) + .addRegularColumn(PER_TABLE_DISK_SIZE, LongType.instance) + .addRegularColumn(PER_COLUMN_DISK_SIZE, LongType.instance) + .build()); + } + + + @Override + public void apply(PartitionUpdate update) + { + // TODO port DataSet. Now we can't change index queryability via system view + throw new InvalidRequestException("Modification is not supported by table " + metadata); + } + + @Override + public DataSet data() + { + SimpleDataSet dataset = new SimpleDataSet(metadata()); + + for (String ks : Schema.instance.getUserKeyspaces()) + { + Keyspace keyspace = Schema.instance.getKeyspaceInstance(ks); + if (keyspace == null) + throw new IllegalArgumentException("Unknown keyspace " + ks); + + for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) + { + SecondaryIndexManager manager = cfs.indexManager; + StorageAttachedIndexGroup group = StorageAttachedIndexGroup.getIndexGroup(cfs); + + if (group != null) + { + for (Index index : group.getIndexes()) + { + ColumnContext context = ((StorageAttachedIndex)index).getContext(); + String indexName = context.getIndexName(); + View view = context.getView(); + + dataset.row(ks, indexName) + .column(TABLE_NAME, cfs.name) + .column(COLUMN_NAME, context.getColumnName()) + .column(IS_QUERYABLE, manager.isIndexQueryable(index)) + .column(IS_BUILDING, manager.isIndexBuilding(indexName)) + .column(IS_STRING, context.isLiteral()) + .column(ANALYZER, context.getAnalyzer().toString()) + .column(INDEXED_SSTABLE_COUNT, view.size()) + .column(CELL_COUNT, context.getCellCount()) + .column(PER_TABLE_DISK_SIZE, group.diskUsage()) + .column(PER_COLUMN_DISK_SIZE, context.diskUsage()); + } + } + } + } + + return dataset; + } + + private static Consumer isQueryableUpdateConsumer(SecondaryIndexManager manager, StorageAttachedIndex index) + { + return isQueryable -> { + logger.debug(index.getContext().logMessage("Index is now {}queryable."), isQueryable ? "" : "non-"); + + if (isQueryable) + manager.makeIndexQueryable(index, Index.Status.BUILD_SUCCEEDED); + else + manager.makeIndexNonQueryable(index, Index.Status.BUILD_FAILED); + }; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/virtual/SSTablesSystemView.java b/src/java/org/apache/cassandra/index/sai/virtual/SSTablesSystemView.java new file mode 100644 index 000000000000..dbbe2b83876c --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/virtual/SSTablesSystemView.java @@ -0,0 +1,131 @@ +/* + * 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.index.sai.virtual; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.virtual.AbstractVirtualTable; +import org.apache.cassandra.db.virtual.SimpleDataSet; +import org.apache.cassandra.db.virtual.VirtualTable; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.index.sai.StorageAttachedIndexGroup; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadata; + +/** + * A {@link VirtualTable} providing a system view of SSTable index metadata. + */ +public class SSTablesSystemView extends AbstractVirtualTable +{ + static final String NAME = "sstable_indexes"; + + static final String KEYSPACE_NAME = "keyspace_name"; + static final String INDEX_NAME = "index_name"; + static final String SSTABLE_NAME = "sstable_name"; + static final String TABLE_NAME = "table_name"; + static final String COLUMN_NAME = "column_name"; + static final String FORMAT_VERSION = "format_version"; + static final String CELL_COUNT = "cell_count"; + static final String MIN_ROW_ID = "min_row_id"; + static final String MAX_ROW_ID = "max_row_id"; + static final String START_TOKEN = "start_token"; + static final String END_TOKEN = "end_token"; + static final String PER_TABLE_DISK_SIZE = "per_table_disk_size"; + static final String PER_COLUMN_DISK_SIZE = "per_column_disk_size"; + + public SSTablesSystemView(String keyspace) + { + super(TableMetadata.builder(keyspace, NAME) + .partitioner(new LocalPartitioner(UTF8Type.instance)) + .comment("SSTable index metadata") + .kind(TableMetadata.Kind.VIRTUAL) + .addPartitionKeyColumn(KEYSPACE_NAME, UTF8Type.instance) + .addClusteringColumn(INDEX_NAME, UTF8Type.instance) + .addClusteringColumn(SSTABLE_NAME, UTF8Type.instance) + .addRegularColumn(TABLE_NAME, UTF8Type.instance) + .addRegularColumn(COLUMN_NAME, UTF8Type.instance) + .addRegularColumn(FORMAT_VERSION, UTF8Type.instance) + .addRegularColumn(CELL_COUNT, LongType.instance) + .addRegularColumn(MIN_ROW_ID, LongType.instance) + .addRegularColumn(MAX_ROW_ID, LongType.instance) + .addRegularColumn(START_TOKEN, UTF8Type.instance) + .addRegularColumn(END_TOKEN, UTF8Type.instance) + .addRegularColumn(PER_TABLE_DISK_SIZE, LongType.instance) + .addRegularColumn(PER_COLUMN_DISK_SIZE, LongType.instance) + .build()); + } + + @Override + public DataSet data() + { + SimpleDataSet dataset = new SimpleDataSet(metadata()); + + for (String ks : Schema.instance.getUserKeyspaces()) + { + Keyspace keyspace = Schema.instance.getKeyspaceInstance(ks); + if (keyspace == null) + throw new IllegalArgumentException("Unknown keyspace " + ks); + + for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) + { + StorageAttachedIndexGroup group = StorageAttachedIndexGroup.getIndexGroup(cfs); + + if (group != null) + { + Token.TokenFactory tokenFactory = cfs.metadata().partitioner.getTokenFactory(); + + for (Index index : group.getIndexes()) + { + ColumnContext columnContext = ((StorageAttachedIndex)index).getContext(); + + for (SSTableIndex sstableIndex : columnContext.getView()) + { + SSTableReader sstable = sstableIndex.getSSTable(); + Descriptor descriptor = sstable.descriptor; + AbstractBounds bounds = sstable.getBounds(); + + dataset.row(ks, columnContext.getIndexName(), sstable.getFilename()) + .column(TABLE_NAME, descriptor.cfname) + .column(COLUMN_NAME, columnContext.getColumnName()) + .column(FORMAT_VERSION, sstableIndex.getVersion().toString()) + .column(CELL_COUNT, sstableIndex.getRowCount()) + .column(MIN_ROW_ID, sstableIndex.minSSTableRowId()) + .column(MAX_ROW_ID, sstableIndex.maxSSTableRowId()) + .column(START_TOKEN, tokenFactory.toString(bounds.left)) + .column(END_TOKEN, tokenFactory.toString(bounds.right)) + .column(PER_TABLE_DISK_SIZE, sstableIndex.getSSTableContext().diskUsage()) + .column(PER_COLUMN_DISK_SIZE, sstableIndex.sizeOfPerColumnComponents()); + } + } + } + } + } + + return dataset; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/virtual/SegmentsSystemView.java b/src/java/org/apache/cassandra/index/sai/virtual/SegmentsSystemView.java new file mode 100644 index 000000000000..5e2b74f62802 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/virtual/SegmentsSystemView.java @@ -0,0 +1,147 @@ +/* + * 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.index.sai.virtual; + +import java.util.List; +import java.util.function.Consumer; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.virtual.AbstractVirtualTable; +import org.apache.cassandra.db.virtual.SimpleDataSet; +import org.apache.cassandra.db.virtual.VirtualTable; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.sai.ColumnContext; +import org.apache.cassandra.index.sai.SSTableIndex; +import org.apache.cassandra.index.sai.StorageAttachedIndex; +import org.apache.cassandra.index.sai.StorageAttachedIndexGroup; +import org.apache.cassandra.index.sai.disk.SegmentMetadata; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadata; + +/** + * A {@link VirtualTable} providing a system view of SSTable index segment metadata. + */ +public class SegmentsSystemView extends AbstractVirtualTable +{ + static final String NAME = "sstable_index_segments"; + + static final String KEYSPACE_NAME = "keyspace_name"; + static final String INDEX_NAME = "index_name"; + static final String SSTABLE_NAME = "sstable_name"; + static final String TABLE_NAME = "table_name"; + static final String COLUMN_NAME = "column_name"; + static final String CELL_COUNT = "cell_count"; + static final String SEGMENT_ROW_ID_OFFSET = "segment_row_id_offset"; + static final String MIN_SSTABLE_ROW_ID = "min_sstable_row_id"; + static final String MAX_SSTABLE_ROW_ID = "max_sstable_row_id"; + static final String START_TOKEN = "start_token"; + static final String END_TOKEN = "end_token"; + static final String MIN_TERM = "min_term"; + static final String MAX_TERM = "max_term"; + static final String COMPONENT_METADATA = "component_metadata"; + + public SegmentsSystemView(String keyspace) + { + super(TableMetadata.builder(keyspace, NAME) + .partitioner(new LocalPartitioner(UTF8Type.instance)) + .comment("SSTable index segment metadata") + .kind(TableMetadata.Kind.VIRTUAL) + .addPartitionKeyColumn(KEYSPACE_NAME, UTF8Type.instance) + .addClusteringColumn(INDEX_NAME, UTF8Type.instance) + .addClusteringColumn(SSTABLE_NAME, UTF8Type.instance) + .addClusteringColumn(SEGMENT_ROW_ID_OFFSET, LongType.instance) + .addRegularColumn(TABLE_NAME, UTF8Type.instance) + .addRegularColumn(COLUMN_NAME, UTF8Type.instance) + .addRegularColumn(CELL_COUNT, LongType.instance) + .addRegularColumn(MIN_SSTABLE_ROW_ID, LongType.instance) + .addRegularColumn(MAX_SSTABLE_ROW_ID, LongType.instance) + .addRegularColumn(START_TOKEN, UTF8Type.instance) + .addRegularColumn(END_TOKEN, UTF8Type.instance) + .addRegularColumn(MIN_TERM, UTF8Type.instance) + .addRegularColumn(MAX_TERM, UTF8Type.instance) + .addRegularColumn(COMPONENT_METADATA, + MapType.getInstance(UTF8Type.instance, + MapType.getInstance(UTF8Type.instance, UTF8Type.instance, false), + false)) + .build()); + } + + @Override + public DataSet data() + { + SimpleDataSet dataset = new SimpleDataSet(metadata()); + + forEachIndex(columnContext -> { + for (SSTableIndex sstableIndex : columnContext.getView()) + { + SSTableReader sstable = sstableIndex.getSSTable(); + List segments = sstableIndex.segments(); + Descriptor descriptor = sstable.descriptor; + Token.TokenFactory tokenFactory = sstable.metadata().partitioner.getTokenFactory(); + + for (SegmentMetadata metadata : segments) + { + dataset.row(sstable.metadata().keyspace, columnContext.getIndexName(), sstable.getFilename(), metadata.segmentRowIdOffset) + .column(TABLE_NAME, descriptor.cfname) + .column(COLUMN_NAME, columnContext.getColumnName()) + .column(CELL_COUNT, metadata.numRows) + .column(MIN_SSTABLE_ROW_ID, metadata.minSSTableRowId) + .column(MAX_SSTABLE_ROW_ID, metadata.maxSSTableRowId) + .column(START_TOKEN, tokenFactory.toString(metadata.minKey.getToken())) + .column(END_TOKEN, tokenFactory.toString(metadata.maxKey.getToken())) + .column(MIN_TERM, columnContext.getValidator().getSerializer().deserialize(metadata.minTerm).toString()) + .column(MAX_TERM, columnContext.getValidator().getSerializer().deserialize(metadata.maxTerm).toString()) + .column(COMPONENT_METADATA, metadata.componentMetadatas.asMap()); + } + } + }); + + return dataset; + } + + private void forEachIndex(Consumer process) + { + for (String ks : Schema.instance.getUserKeyspaces()) + { + Keyspace keyspace = Schema.instance.getKeyspaceInstance(ks); + if (keyspace == null) + throw new IllegalArgumentException("Unknown keyspace " + ks); + + for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) + { + StorageAttachedIndexGroup group = StorageAttachedIndexGroup.getIndexGroup(cfs); + + if (group != null) + { + for (Index index : group.getIndexes()) + { + process.accept(((StorageAttachedIndex)index).getContext()); + } + } + } + } + } +} diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java index b1998bc40b5b..c07a395416dd 100644 --- a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java +++ b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java @@ -30,8 +30,10 @@ import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.db.lifecycle.Tracker; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.partitions.PartitionIterator; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.Row; @@ -46,7 +48,7 @@ import org.apache.cassandra.index.sasi.conf.IndexMode; import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.Mode; import org.apache.cassandra.index.sasi.disk.PerSSTableIndexWriter; -import org.apache.cassandra.index.sasi.plan.QueryPlan; +import org.apache.cassandra.index.sasi.plan.SASIIndexSearcher; import org.apache.cassandra.index.transactions.IndexTransaction; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; @@ -70,7 +72,8 @@ private static class SASIIndexBuildingSupport implements IndexBuildingSupport { public SecondaryIndexBuilder getIndexBuildTask(ColumnFamilyStore cfs, Set indexes, - Collection sstablesToRebuild) + Collection sstablesToRebuild, + boolean isFullRebuild) { NavigableMap> sstables = new TreeMap<>((a, b) -> { return Integer.compare(a.descriptor.generation, b.descriptor.generation); @@ -116,7 +119,7 @@ public SASIIndex(ColumnFamilyStore baseCfs, IndexMetadata config) SortedMap> toRebuild = new TreeMap<>((a, b) -> Integer.compare(a.descriptor.generation, b.descriptor.generation)); - for (SSTableReader sstable : index.init(tracker.getView().liveSSTables())) + for (SSTableReader sstable : index.init(tracker.getLiveSSTables())) { Map perSSTable = toRebuild.get(sstable); if (perSSTable == null) @@ -255,7 +258,7 @@ public boolean supportsReplicaFilteringProtection(RowFilter rowFilter) return false; } - public Indexer indexerFor(DecoratedKey key, RegularAndStaticColumns columns, int nowInSec, WriteContext context, IndexTransaction.Type transactionType) + public Indexer indexerFor(DecoratedKey key, RegularAndStaticColumns columns, int nowInSec, WriteContext context, IndexTransaction.Type transactionType, Memtable memtable) { return new Indexer() { @@ -294,7 +297,7 @@ private boolean isNewData() public void adjustMemtableSize(long additionalSpace, OpOrder.Group opGroup) { - baseCfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().allocate(additionalSpace, opGroup); + baseCfs.getTracker().getView().getCurrentMemtable().markExtraOnHeapUsed(additionalSpace, opGroup); } }; } @@ -303,17 +306,12 @@ public Searcher searcherFor(ReadCommand command) throws InvalidRequestException { TableMetadata config = command.metadata(); ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(config.id); - return controller -> new QueryPlan(cfs, command, DatabaseDescriptor.getRangeRpcTimeout(MILLISECONDS)).execute(controller); + return new SASIIndexSearcher(cfs, command, DatabaseDescriptor.getRangeRpcTimeout(MILLISECONDS)); } - public SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationType opType) + public SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker) { - return newWriter(baseCfs.metadata().partitionKeyType, descriptor, Collections.singletonMap(index.getDefinition(), index), opType); - } - - public BiFunction postProcessorFor(ReadCommand command) - { - return (partitionIterator, readCommand) -> partitionIterator; + return newWriter(baseCfs.metadata().partitionKeyType, descriptor, Collections.singletonMap(index.getDefinition(), index), tracker.opType()); } public IndexBuildingSupport getBuildTaskSupport() diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java index bb42dc2d178a..8cbb9a12cd7e 100644 --- a/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java +++ b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java @@ -22,13 +22,14 @@ import java.io.File; import java.io.IOException; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.SortedMap; +import java.util.UUID; -import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.RowIndexEntry; -import org.apache.cassandra.db.compaction.CompactionInfo; import org.apache.cassandra.db.compaction.CompactionInterruptedException; import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.marshal.AbstractType; @@ -36,11 +37,12 @@ import org.apache.cassandra.index.sasi.conf.ColumnIndex; import org.apache.cassandra.index.sasi.disk.PerSSTableIndexWriter; import org.apache.cassandra.io.FSReadError; -import org.apache.cassandra.io.sstable.KeyIterator; -import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableIdentityIterator; +import org.apache.cassandra.io.sstable.format.PartitionIndexIterator; +import org.apache.cassandra.io.sstable.format.RowIndexEntry; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.UUIDGen; @@ -58,7 +60,7 @@ public SASIIndexBuilder(ColumnFamilyStore cfs, SortedMap indexes) diff --git a/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java index c67c39c645bc..f4bc0884a8ad 100644 --- a/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java +++ b/src/java/org/apache/cassandra/index/sasi/SSTableIndex.java @@ -33,7 +33,9 @@ import org.apache.cassandra.index.sasi.utils.RangeIterator; import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.io.util.RandomAccessReader; import org.apache.cassandra.utils.concurrent.Ref; import org.apache.commons.lang3.builder.HashCodeBuilder; @@ -43,7 +45,7 @@ public class SSTableIndex { private final ColumnIndex columnIndex; - private final Ref sstableRef; + private final Ref sstableRef; private final SSTableReader sstable; private final OnDiskIndex index; private final AtomicInteger references = new AtomicInteger(1); @@ -174,9 +176,9 @@ private static class DecoratedKeyFetcher implements Function public DecoratedKey apply(Long offset) { - try + try (RandomAccessReader in = sstable.openKeyComponentReader()) { - return sstable.keyAt(offset); + return sstable.keyAt(in, offset); } catch (IOException e) { diff --git a/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java b/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java index 4c9c59e19916..fc24ce341456 100644 --- a/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java +++ b/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java @@ -31,7 +31,7 @@ import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.Memtable; +import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.UTF8Type; diff --git a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java index 0af4ba2aef8e..a9e58ba7fa53 100644 --- a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java +++ b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java @@ -27,6 +27,7 @@ import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.db.DeletionTime; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.compaction.OperationType; @@ -111,6 +112,18 @@ public void startPartition(DecoratedKey key, long curPosition) currentKeyPosition = curPosition; } + @Override + public void partitionLevelDeletion(DeletionTime deletionTime, long position) + { + // do nothing + } + + @Override + public void staticRow(Row staticRow, long position) + { + nextUnfilteredCluster(staticRow); + } + public void nextUnfilteredCluster(Unfiltered unfiltered) { if (!unfiltered.isRow()) diff --git a/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java b/src/java/org/apache/cassandra/index/sasi/plan/SASIIndexSearcher.java similarity index 92% rename from src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java rename to src/java/org/apache/cassandra/index/sasi/plan/SASIIndexSearcher.java index a54dfc8dd588..a613aabcf735 100644 --- a/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java +++ b/src/java/org/apache/cassandra/index/sasi/plan/SASIIndexSearcher.java @@ -23,19 +23,21 @@ import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.index.Index; import org.apache.cassandra.index.sasi.disk.Token; import org.apache.cassandra.index.sasi.plan.Operation.OperationType; -import org.apache.cassandra.exceptions.RequestTimeoutException; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.AbstractIterator; -public class QueryPlan +public class SASIIndexSearcher implements Index.Searcher { + private final ReadCommand command; private final QueryController controller; - public QueryPlan(ColumnFamilyStore cfs, ReadCommand command, long executionQuotaMs) + public SASIIndexSearcher(ColumnFamilyStore cfs, ReadCommand command, long executionQuotaMs) { + this.command = command; this.controller = new QueryController(cfs, (PartitionRangeReadCommand) command, executionQuotaMs); } @@ -63,7 +65,14 @@ private Operation analyze() } } - public UnfilteredPartitionIterator execute(ReadExecutionController executionController) throws RequestTimeoutException + @Override + public ReadCommand command() + { + return command; + } + + @Override + public UnfilteredPartitionIterator search(ReadExecutionController executionController) { return new ResultIterator(analyze(), controller, executionController); } diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java index 219082482a19..6496824b8f37 100644 --- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java +++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java @@ -81,12 +81,12 @@ public CompressedSequentialWriter(File file, MetadataCollector sstableMetadataCollector) { super(file, SequentialWriterOption.newBuilder() - .bufferSize(option.bufferSize()) - .bufferType(option.bufferType()) - .bufferSize(parameters.chunkLength()) - .bufferType(parameters.getSstableCompressor().preferredBufferType()) - .finishOnClose(option.finishOnClose()) - .build()); + .bufferSize(option.bufferSize()) + .bufferType(option.bufferType()) + .bufferSize(parameters.chunkLength()) + .bufferType(parameters.getSstableCompressor().preferredBufferType()) + .finishOnClose(option.finishOnClose()) + .build()); this.compressor = parameters.getSstableCompressor(); this.digestFile = Optional.ofNullable(digestFile); @@ -333,6 +333,41 @@ private void seekToChunkStart() } } + // Page management using chunk boundaries + + @Override + public int maxBytesInPage() + { + return buffer.capacity(); + } + + @Override + public void padToPageBoundary() throws IOException + { + if (buffer.position() == 0) + return; + + int padLength = buffer.remaining(); + + // Flush as much as we have + doFlush(0); + // But pretend we had a whole chunk + bufferOffset += padLength; + lastFlushOffset += padLength; + } + + @Override + public int bytesLeftInPage() + { + return buffer.remaining(); + } + + @Override + public long paddedPosition() + { + return position() + (buffer.position() == 0 ? 0 : buffer.remaining()); + } + protected class TransactionalProxy extends SequentialWriter.TransactionalProxy { @Override diff --git a/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java b/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java index 07523a001a92..801d7ed8d4c4 100644 --- a/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java +++ b/src/java/org/apache/cassandra/io/sstable/BloomFilterTracker.java @@ -17,83 +17,147 @@ */ package org.apache.cassandra.io.sstable; -import java.util.concurrent.atomic.AtomicLong; +import com.codahale.metrics.Meter; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.io.sstable.format.SSTableReader; -public class BloomFilterTracker +public abstract class BloomFilterTracker { - private final AtomicLong falsePositiveCount = new AtomicLong(0); - private final AtomicLong truePositiveCount = new AtomicLong(0); - private final AtomicLong trueNegativeCount = new AtomicLong(0); - private long lastFalsePositiveCount = 0L; - private long lastTruePositiveCount = 0L; - private long lastTrueNegativeCount = 0L; - - public void addFalsePositive() - { - falsePositiveCount.incrementAndGet(); - } + public abstract void addFalsePositive(); + public abstract void addTruePositive(); + public abstract void addTrueNegative(); + public abstract long getFalsePositiveCount(); + public abstract double getRecentFalsePositiveRate(); + public abstract long getTruePositiveCount(); + public abstract double getRecentTruePositiveRate(); + public abstract long getTrueNegativeCount(); + public abstract double getRecentTrueNegativeRate(); - public void addTruePositive() + public static BloomFilterTracker createNoopTracker() { - truePositiveCount.incrementAndGet(); + return NoopBloomFilterTracker.instance; } - public void addTrueNegative() + public static BloomFilterTracker createMeterTracker() { - trueNegativeCount.incrementAndGet(); + return new MeterBloomFilterTracker(); } - public long getFalsePositiveCount() + private static class MeterBloomFilterTracker extends BloomFilterTracker { - return falsePositiveCount.get(); - } + private final Meter falsePositiveCount = new Meter(); + private final Meter truePositiveCount = new Meter(); + private final Meter trueNegativeCount = new Meter(); - public long getRecentFalsePositiveCount() - { - long fpc = getFalsePositiveCount(); - try + @Override + public void addFalsePositive() { - return (fpc - lastFalsePositiveCount); + falsePositiveCount.mark(); } - finally + + @Override + public void addTruePositive() { - lastFalsePositiveCount = fpc; + truePositiveCount.mark(); } - } - public long getTruePositiveCount() - { - return truePositiveCount.get(); - } + @Override + public void addTrueNegative() + { + trueNegativeCount.mark(); + } - public long getRecentTruePositiveCount() - { - long tpc = getTruePositiveCount(); - try + @Override + public long getFalsePositiveCount() { - return (tpc - lastTruePositiveCount); + return falsePositiveCount.getCount(); } - finally + + @Override + public double getRecentFalsePositiveRate() { - lastTruePositiveCount = tpc; + return falsePositiveCount.getFifteenMinuteRate(); } - } - public long getTrueNegativeCount() - { - return trueNegativeCount.get(); + @Override + public long getTruePositiveCount() + { + return truePositiveCount.getCount(); + } + + @Override + public double getRecentTruePositiveRate() + { + return truePositiveCount.getFifteenMinuteRate(); + } + + @Override + public long getTrueNegativeCount() + { + return trueNegativeCount.getCount(); + } + + @Override + public double getRecentTrueNegativeRate() + { + return trueNegativeCount.getFifteenMinuteRate(); + } } - public long getRecentTrueNegativeCount() + /** + * Bloom filter tracker that does nothing and always returns 0 for all counters. + * + * Bloom Filter tracking is managed on the CFS level, so there is no reason to count anything if an SSTable does not + * belong (yet) to a CFS. This tracker is used initially on SSTableReaders and is overwritten during setup + * in {@link SSTableReader#setupOnline()} or {@link SSTableReader#setupOnline(ColumnFamilyStore)}}. + */ + private static class NoopBloomFilterTracker extends BloomFilterTracker { - long tnc = getTrueNegativeCount(); - try + static final NoopBloomFilterTracker instance = new NoopBloomFilterTracker(); + + @Override + public void addFalsePositive() {} + + @Override + public void addTruePositive() {} + + @Override + public void addTrueNegative() {} + + @Override + public long getFalsePositiveCount() + { + return 0; + } + + @Override + public double getRecentFalsePositiveRate() { - return (tnc - lastTrueNegativeCount); + return 0; } - finally + + @Override + public long getTruePositiveCount() + { + return 0; + } + + @Override + public double getRecentTruePositiveRate() + { + return 0; + } + + @Override + public long getTrueNegativeCount() + { + return 0; + } + + @Override + public double getRecentTrueNegativeRate() { - lastTrueNegativeCount = tnc; + return 0; } } } diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java index 0ac189c9e77e..999e1bd11f2c 100644 --- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java @@ -50,7 +50,7 @@ import org.apache.cassandra.exceptions.SyntaxException; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.schema.*; -import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.utils.ByteBufferUtil; @@ -235,20 +235,21 @@ public CQLSSTableWriter rawAddRow(List values) if (values.size() != boundNames.size()) throw new InvalidRequestException(String.format("Invalid number of arguments, expecting %d values but got %d", boundNames.size(), values.size())); + QueryState state = QueryState.forInternalCalls(); QueryOptions options = QueryOptions.forInternalCalls(null, values); - List keys = insert.buildPartitionKeyNames(options); - SortedSet> clusterings = insert.createClustering(options); + List keys = insert.buildPartitionKeyNames(options, state); + SortedSet> clusterings = insert.createClustering(options, state); long now = System.currentTimeMillis(); // Note that we asks indexes to not validate values (the last 'false' arg below) because that triggers a 'Keyspace.open' // and that forces a lot of initialization that we don't want. UpdateParameters params = new UpdateParameters(insert.metadata, insert.updatedColumns(), + QueryState.forInternalCalls(), options, insert.getTimestamp(TimeUnit.MILLISECONDS.toMicros(now), options), (int) TimeUnit.MILLISECONDS.toSeconds(now), - insert.getTimeToLive(options), - Collections.emptyMap()); + insert.getTimeToLive(options), Collections.emptyMap()); try { @@ -558,9 +559,9 @@ private Types createTypes(String keyspace) */ private TableMetadata createTable(Types types) { - ClientState state = ClientState.forInternalCalls(); - CreateTableStatement statement = schemaStatement.prepare(state); - statement.validate(ClientState.forInternalCalls()); + QueryState state = QueryState.forInternalCalls(); + CreateTableStatement statement = schemaStatement.prepare(state.getClientState()); + statement.validate(state); TableMetadata.Builder builder = statement.builder(types); if (partitioner != null) @@ -576,8 +577,8 @@ private TableMetadata createTable(Types types) */ private UpdateStatement prepareInsert() { - ClientState state = ClientState.forInternalCalls(); - UpdateStatement insert = (UpdateStatement) insertStatement.prepare(state); + QueryState state = QueryState.forInternalCalls(); + UpdateStatement insert = (UpdateStatement) insertStatement.prepare(state.getClientState()); insert.validate(state); if (insert.hasConditions()) diff --git a/src/java/org/apache/cassandra/io/sstable/Component.java b/src/java/org/apache/cassandra/io/sstable/Component.java index a81db859f23d..aba9ea21a474 100644 --- a/src/java/org/apache/cassandra/io/sstable/Component.java +++ b/src/java/org/apache/cassandra/io/sstable/Component.java @@ -42,6 +42,10 @@ public enum Type // the base data for an sstable: the remaining components can be regenerated // based on the data component DATA("Data.db"), + // partition index trie (TrieIndexFormat) + PARTITION_INDEX("Partitions.db"), + // row indices (TrieIndexFormat) + ROW_INDEX("Rows.db"), // index of the row keys with pointers to their positions in the data file PRIMARY_INDEX("Index.db"), // serialized bloom filter for the row keys in the sstable @@ -83,6 +87,8 @@ static Type fromRepresentation(String repr) // singleton components for types that don't need ids public final static Component DATA = new Component(Type.DATA); + public final static Component PARTITION_INDEX = new Component(Type.PARTITION_INDEX); + public final static Component ROW_INDEX = new Component(Type.ROW_INDEX); public final static Component PRIMARY_INDEX = new Component(Type.PRIMARY_INDEX); public final static Component FILTER = new Component(Type.FILTER); public final static Component COMPRESSION_INFO = new Component(Type.COMPRESSION_INFO); @@ -133,6 +139,8 @@ public static Component parse(String name) switch (type) { case DATA: return Component.DATA; + case PARTITION_INDEX: return Component.PARTITION_INDEX; + case ROW_INDEX: return Component.ROW_INDEX; case PRIMARY_INDEX: return Component.PRIMARY_INDEX; case FILTER: return Component.FILTER; case COMPRESSION_INFO: return Component.COMPRESSION_INFO; diff --git a/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java b/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java index 93be2eec6a7e..d4ed3def4259 100644 --- a/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java +++ b/src/java/org/apache/cassandra/io/sstable/CorruptSSTableException.java @@ -33,4 +33,10 @@ public CorruptSSTableException(Throwable cause, String path) { this(cause, new File(path)); } + + protected CorruptSSTableException(String msg, Throwable cause, File path) + { + super(msg, cause); + this.path = path; + } } diff --git a/src/java/org/apache/cassandra/io/sstable/Descriptor.java b/src/java/org/apache/cassandra/io/sstable/Descriptor.java index b781ebf50cd5..31becad06a15 100644 --- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java +++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java @@ -116,6 +116,11 @@ public Descriptor withFormatType(SSTableFormat.Type newType) return new Descriptor(newType.info.getLatestVersion(), directory, ksname, cfname, generation, newType); } + public File tmpFileFor(Component component) + { + return new File(tmpFilenameFor(component)); + } + public String tmpFilenameFor(Component component) { return filenameFor(component) + TMP_EXT; @@ -131,6 +136,11 @@ public String tmpFilenameForStreaming(Component component) return String.format("%s.%s%s", filenameFor(component), UUIDGen.getTimeUUID(), TMP_EXT); } + public File fileFor(Component component) + { + return new File(filenameFor(component)); + } + public String filenameFor(Component component) { return baseFilename() + separator + component.name(); diff --git a/src/java/org/apache/cassandra/io/sstable/ISSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/ISSTableScanner.java index af661b7d66c9..0f1897ded8be 100644 --- a/src/java/org/apache/cassandra/io/sstable/ISSTableScanner.java +++ b/src/java/org/apache/cassandra/io/sstable/ISSTableScanner.java @@ -34,13 +34,14 @@ */ public interface ISSTableScanner extends UnfilteredPartitionIterator { - public long getLengthInBytes(); - public long getCompressedLengthInBytes(); - public long getCurrentPosition(); - public long getBytesScanned(); - public Set getBackingSSTables(); + long getLengthInBytes(); + long getCompressedLengthInBytes(); + long getCurrentPosition(); + long getBytesScanned(); + Set getBackingSSTables(); + int level(); - public static void closeAllAndPropagate(Collection scanners, Throwable throwable) + static void closeAllAndPropagate(Collection scanners, Throwable throwable) { for (ISSTableScanner scanner: scanners) { @@ -66,6 +67,5 @@ public static void closeAllAndPropagate(Collection scanners, Th { Throwables.propagate(throwable); } - } } diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java index a85d855abdb9..900f811acc41 100644 --- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java @@ -186,7 +186,8 @@ private List getAllSSTables() for (Keyspace ks : Keyspace.all()) { for (ColumnFamilyStore cfStore: ks.getColumnFamilyStores()) - result.addAll(cfStore.getLiveSSTables()); + for (SSTableReader tr : SSTableReader.selectOnlyBigTableReaders(cfStore.getLiveSSTables())) + result.add(tr); } return result; @@ -213,8 +214,8 @@ private Pair> getRestributionTransactio do { View view = cfStore.getTracker().getView(); - allSSTables = ImmutableSet.copyOf(view.select(SSTableSet.CANONICAL)); - nonCompacting = ImmutableSet.copyOf(view.getUncompacting(allSSTables)); + allSSTables = ImmutableSet.copyOf(SSTableReader.selectOnlyBigTableReaders(view.select(SSTableSet.CANONICAL))); + nonCompacting = ImmutableSet.copyOf(SSTableReader.selectOnlyBigTableReaders(view.getNoncompacting(allSSTables))); } while (null == (txn = cfStore.getTracker().tryModify(nonCompacting, OperationType.INDEX_SUMMARY))); diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java index 90a86215566d..cd930108dc7a 100644 --- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; @@ -33,10 +34,9 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.compaction.CompactionInfo; +import org.apache.cassandra.db.compaction.AbstractTableOperation; import org.apache.cassandra.db.compaction.CompactionInterruptedException; import org.apache.cassandra.db.compaction.OperationType; -import org.apache.cassandra.db.compaction.CompactionInfo.Unit; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.metrics.StorageMetrics; @@ -47,7 +47,7 @@ import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL; -public class IndexSummaryRedistribution extends CompactionInfo.Holder +public class IndexSummaryRedistribution extends AbstractTableOperation { private static final Logger logger = LoggerFactory.getLogger(IndexSummaryRedistribution.class); @@ -84,7 +84,7 @@ public List redistributeSummaries() throws IOException List redistribute = new ArrayList<>(); for (LifecycleTransaction txn : transactions.values()) { - redistribute.addAll(txn.originals()); + redistribute.addAll(SSTableReader.selectOnlyBigTableReaders(txn.originals(), Collectors.toList())); } long total = nonRedistributingOffHeapSize; @@ -99,7 +99,7 @@ public List redistributeSummaries() throws IOException for (SSTableReader sstable : redistribute) { if (isStopRequested()) - throw new CompactionInterruptedException(getCompactionInfo()); + throw new CompactionInterruptedException(getProgress()); if (sstable.getReadMeter() != null) { @@ -119,7 +119,7 @@ public List redistributeSummaries() throws IOException logger.trace("Index summaries for compacting SSTables are using {} MB of space", (memoryPoolBytes - remainingBytes) / 1024.0 / 1024.0); List newSSTables; - try (Refs refs = Refs.ref(sstablesByHotness)) + try (Refs refs = Refs.ref(sstablesByHotness)) { newSSTables = adjustSamplingLevels(sstablesByHotness, transactions, totalReadsPerSec, remainingBytes); @@ -152,7 +152,7 @@ private List adjustSamplingLevels(List sstables, for (SSTableReader sstable : sstables) { if (isStopRequested()) - throw new CompactionInterruptedException(getCompactionInfo()); + throw new CompactionInterruptedException(getProgress()); int minIndexInterval = sstable.metadata().params.minIndexInterval; int maxIndexInterval = sstable.metadata().params.maxIndexInterval; @@ -249,7 +249,7 @@ else if (targetNumEntries < currentNumEntries * DOWNSAMPLE_THESHOLD && newSampli for (ResampleEntry entry : toDownsample) { if (isStopRequested()) - throw new CompactionInterruptedException(getCompactionInfo()); + throw new CompactionInterruptedException(getProgress()); SSTableReader sstable = entry.sstable; logger.trace("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries", @@ -328,9 +328,14 @@ public int compare(ResampleEntry o1, ResampleEntry o2) return Pair.create(willNotDownsample, toDownsample.subList(noDownsampleCutoff, toDownsample.size())); } - public CompactionInfo getCompactionInfo() + public OperationProgress getProgress() { - return CompactionInfo.withoutSSTables(null, OperationType.INDEX_SUMMARY, (memoryPoolBytes - remainingSpace), memoryPoolBytes, Unit.BYTES, compactionId); + return OperationProgress.withoutSSTables(null, + OperationType.INDEX_SUMMARY, + (memoryPoolBytes - remainingSpace), + memoryPoolBytes, + Unit.BYTES, + compactionId); } public boolean isGlobal() diff --git a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java index 1a5792c4fad2..475a6c9f03a3 100644 --- a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java +++ b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java @@ -17,105 +17,64 @@ */ package org.apache.cassandra.io.sstable; -import java.io.File; import java.io.IOException; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.RowIndexEntry; import org.apache.cassandra.dht.IPartitioner; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.RandomAccessReader; -import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.io.sstable.format.PartitionIndexIterator; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.utils.AbstractIterator; -import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.CloseableIterator; +// TODO STAR-247: Implement a unit test public class KeyIterator extends AbstractIterator implements CloseableIterator { - private final static class In - { - private final File path; - private volatile RandomAccessReader in; - - public In(File path) - { - this.path = path; - } - - private void maybeInit() - { - if (in != null) - return; - - synchronized (this) - { - if (in == null) - { - in = RandomAccessReader.open(path); - } - } - } - - public DataInputPlus get() - { - maybeInit(); - return in; - } - - public boolean isEOF() - { - maybeInit(); - return in.isEOF(); - } - - public void close() - { - if (in != null) - in.close(); - } + private final IPartitioner partitioner; + private final PartitionIndexIterator it; + private final ReadWriteLock fileAccessLock; + private final long totalBytes; - public long getFilePointer() - { - maybeInit(); - return in.getFilePointer(); - } + private boolean initialized = false; - public long length() - { - maybeInit(); - return in.length(); - } + public KeyIterator(PartitionIndexIterator it, IPartitioner partitioner, long totalBytes, ReadWriteLock fileAccessLock) + { + this.it = it; + this.partitioner = partitioner; + this.totalBytes = totalBytes; + this.fileAccessLock = fileAccessLock; } - private final Descriptor desc; - private final In in; - private final IPartitioner partitioner; - private final ReadWriteLock fileAccessLock; - - private long keyPosition; + public KeyIterator(PartitionIndexIterator it, IPartitioner partitioner, long totalBytes) + { + this(it, partitioner, totalBytes, null); + } - public KeyIterator(Descriptor desc, TableMetadata metadata) + public static KeyIterator forSSTable(SSTableReader ssTableReader) throws IOException { - this.desc = desc; - in = new In(new File(desc.filenameFor(Component.PRIMARY_INDEX))); - partitioner = metadata.partitioner; - fileAccessLock = new ReentrantReadWriteLock(); + return new KeyIterator(ssTableReader.allKeysIterator(), ssTableReader.getPartitioner(), ssTableReader.uncompressedLength(), new ReentrantReadWriteLock()); } protected DecoratedKey computeNext() { - fileAccessLock.readLock().lock(); + if (fileAccessLock != null) + fileAccessLock.readLock().lock(); try { - if (in.isEOF()) - return endOfData(); - - keyPosition = in.getFilePointer(); - DecoratedKey key = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(in.get())); - RowIndexEntry.Serializer.skip(in.get(), desc.version); // skip remainder of the entry - return key; + if (!initialized) + { + initialized = true; + return it.isExhausted() + ? endOfData() + : partitioner.decorateKey(it.key()); + } + else + { + return it.advance() + ? partitioner.decorateKey(it.key()) + : endOfData(); + } } catch (IOException e) { @@ -123,45 +82,43 @@ protected DecoratedKey computeNext() } finally { - fileAccessLock.readLock().unlock(); + if (fileAccessLock != null) + fileAccessLock.readLock().unlock(); } } public void close() { - fileAccessLock.writeLock().lock(); + if (fileAccessLock != null) + fileAccessLock.writeLock().lock(); try { - in.close(); + it.close(); } finally { - fileAccessLock.writeLock().unlock(); + if (fileAccessLock != null) + fileAccessLock.writeLock().unlock(); } } public long getBytesRead() { - fileAccessLock.readLock().lock(); + if (fileAccessLock != null) + fileAccessLock.readLock().lock(); try { - return in.getFilePointer(); + return it.isExhausted() ? totalBytes : it.dataPosition(); } finally { - fileAccessLock.readLock().unlock(); + if (fileAccessLock != null) + fileAccessLock.readLock().unlock(); } } public long getTotalBytes() { - // length is final in the referenced object. - // no need to acquire the lock - return in.length(); - } - - public long getKeyPosition() - { - return keyPosition; + return totalBytes; } } diff --git a/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java b/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java index 826b91d65257..8c240c8cef87 100644 --- a/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java +++ b/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.io.sstable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; @@ -26,6 +27,7 @@ import org.apache.cassandra.utils.CloseableIterator; import org.apache.cassandra.utils.IMergeIterator; import org.apache.cassandra.utils.MergeIterator; +import org.apache.cassandra.utils.Throwables; /** * Caller must acquire and release references to the sstables used here. @@ -34,12 +36,27 @@ public class ReducingKeyIterator implements CloseableIterator { private final ArrayList iters; private volatile IMergeIterator mi; + private final long totalLength; public ReducingKeyIterator(Collection sstables) { iters = new ArrayList<>(sstables.size()); - for (SSTableReader sstable : sstables) - iters.add(new KeyIterator(sstable.descriptor, sstable.metadata())); + long len = 0; + try + { + for (SSTableReader sstable : sstables) + { + KeyIterator iter = KeyIterator.forSSTable(sstable); + iters.add(iter); + len += iter.getTotalBytes(); + } + } + catch (IOException | RuntimeException ex) + { + iters.forEach(KeyIterator::close); + throw Throwables.cleaned(ex); + } + this.totalLength = len; } private void maybeInit() @@ -83,14 +100,7 @@ public void close() public long getTotalBytes() { - maybeInit(); - - long m = 0; - for (Iterator iter : mi.iterators()) - { - m += ((KeyIterator) iter).getTotalBytes(); - } - return m; + return totalLength; } public long getBytesRead() diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java index 0471be3238cf..82dbe22dbacc 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTable.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java @@ -23,6 +23,7 @@ import java.util.*; import java.util.concurrent.CopyOnWriteArraySet; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import com.google.common.collect.Collections2; @@ -33,7 +34,8 @@ import org.apache.cassandra.db.BufferDecoratedKey; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.RowIndexEntry; +import org.apache.cassandra.db.lifecycle.Tracker; +import org.apache.cassandra.io.sstable.format.PartitionIndexIterator; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; @@ -71,7 +73,7 @@ public abstract class SSTable public static final int TOMBSTONE_HISTOGRAM_TTL_ROUND_SECONDS = Integer.valueOf(System.getProperty("cassandra.streaminghistogram.roundseconds", "60")); public final Descriptor descriptor; - protected final Set components; + public final Set components; public final boolean compression; public DecoratedKey first; @@ -79,6 +81,8 @@ public abstract class SSTable protected final DiskOptimizationStrategy optimizationStrategy; protected final TableMetadataRef metadata; + private static final int SAMPLES_CAP = 10000; + private static final int BYTES_CAP = 10000000; protected SSTable(Descriptor descriptor, Set components, TableMetadataRef metadata, DiskOptimizationStrategy optimizationStrategy) { @@ -157,11 +161,6 @@ public String getFilename() return descriptor.filenameFor(Component.DATA); } - public String getIndexFilename() - { - return descriptor.filenameFor(Component.PRIMARY_INDEX); - } - public String getColumnFamilyName() { return descriptor.cfname; @@ -172,6 +171,11 @@ public String getKeyspaceName() return descriptor.ksname; } + public int getGeneration() + { + return descriptor.generation; + } + public List getAllFilePaths() { List ret = new ArrayList<>(components.size()); @@ -265,21 +269,24 @@ public static Set discoverComponentsFor(Descriptor desc) } /** @return An estimate of the number of keys contained in the given index file. */ - public static long estimateRowsFromIndex(RandomAccessReader ifile, Descriptor descriptor) throws IOException + public static long estimateRowsFromIndex(PartitionIndexIterator iterator) throws IOException { // collect sizes for the first 10000 keys, or first 10 megabytes of data - final int SAMPLES_CAP = 10000, BYTES_CAP = (int)Math.min(10000000, ifile.length()); - int keys = 0; - while (ifile.getFilePointer() < BYTES_CAP && keys < SAMPLES_CAP) + try + { + int keys = 0; + while (!iterator.isExhausted() && iterator.indexPosition() < BYTES_CAP && keys < SAMPLES_CAP) + { + iterator.advance(); + keys++; + } + assert keys > 0 && iterator.indexPosition() > 0 && iterator.indexLength() > 0 : "Unexpected empty index file"; + return iterator.indexLength() / (iterator.indexPosition() / keys); + } + finally { - ByteBufferUtil.skipShortLength(ifile); - RowIndexEntry.Serializer.skip(ifile, descriptor.version); - keys++; + iterator.reset(); } - assert keys > 0 && ifile.getFilePointer() > 0 && ifile.length() > 0 : "Unexpected empty index file: " + ifile; - long estimatedRows = ifile.length() / (ifile.getFilePointer() / keys); - ifile.seek(0); - return estimatedRows; } public long bytesOnDisk() @@ -304,7 +311,8 @@ public String toString() * Reads the list of components from the TOC component. * @return set of components found in the TOC */ - protected static Set readTOC(Descriptor descriptor) throws IOException + @VisibleForTesting + public static Set readTOC(Descriptor descriptor) throws IOException { return readTOC(descriptor, true); } @@ -330,6 +338,17 @@ protected static Set readTOC(Descriptor descriptor, boolean skipMissi return components; } + /** + * Rewrite TOC components by deleting existing TOC file and append new components + */ + private static void rewriteTOC(Descriptor descriptor, Collection components) + { + File tocFile = descriptor.fileFor(Component.TOC); + if (!tocFile.delete()) + logger.error("Failed to delete TOC component for " + descriptor); + appendTOC(descriptor, components); + } + /** * Appends new component names to the TOC component. */ @@ -355,9 +374,48 @@ protected static void appendTOC(Descriptor descriptor, Collection com */ public synchronized void addComponents(Collection newComponents) { - Collection componentsToAdd = Collections2.filter(newComponents, Predicates.not(Predicates.in(components))); + registerComponents(newComponents, null); + } + + /** + * Registers new custom components into sstable and update size tracking + * @param newComponents collection of components to be added + * @param tracker used to update on-disk size metrics + */ + public synchronized void registerComponents(Collection newComponents, Tracker tracker) + { + Collection componentsToAdd = new HashSet<>(Collections2.filter(newComponents, x -> !components.contains(x))); appendTOC(descriptor, componentsToAdd); components.addAll(componentsToAdd); + + if (tracker == null) + return; + + for (Component component : componentsToAdd) + { + File file = descriptor.fileFor(component); + if (file.exists()) + tracker.updateSizeTracking(file.length()); + } + } + + /** + * Unregisters custom components from sstable and update size tracking + * @param removeComponents collection of components to be remove + * @param tracker used to update on-disk size metrics + */ + public synchronized void unregisterComponents(Collection removeComponents, Tracker tracker) + { + Collection componentsToRemove = new HashSet<>(Collections2.filter(removeComponents, components::contains)); + components.removeAll(componentsToRemove); + rewriteTOC(descriptor, components); + + for (Component component : componentsToRemove) + { + File file = descriptor.fileFor(component); + if (file.exists()) + tracker.updateSizeTracking(-file.length()); + } } public AbstractBounds getBounds() @@ -373,4 +431,14 @@ public static void validateRepairedMetadata(long repairedAt, UUID pendingRepair, "isTransient can only be true for sstables pending repair"); } + + public DecoratedKey getFirst() + { + return first; + } + + public DecoratedKey getLast() + { + return last; + } } diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java index 76e12c891ada..367d0c9fa1c0 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java @@ -19,6 +19,7 @@ import java.io.*; +import org.apache.cassandra.io.sstable.format.RowIndexEntry; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.db.*; import org.apache.cassandra.db.rows.*; @@ -69,13 +70,16 @@ public static SSTableIdentityIterator create(SSTableReader sstable, RandomAccess } @SuppressWarnings("resource") - public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInput dfile, RowIndexEntry indexEntry, DecoratedKey key, boolean tombstoneOnly) + public static SSTableIdentityIterator create(SSTableReader sstable, FileDataInput dfile, RowIndexEntry indexEntry, DecoratedKey key, boolean tombstoneOnly) { try { dfile.seek(indexEntry.position); ByteBufferUtil.skipShortLength(dfile); // Skip partition key DeletionTime partitionLevelDeletion = DeletionTime.serializer.deserialize(dfile); + if (!partitionLevelDeletion.validate()) + UnfilteredValidation.handleInvalid(sstable.metadata(), key, sstable, "partitionLevelDeletion="+partitionLevelDeletion.toString()); + DeserializationHelper helper = new DeserializationHelper(sstable.metadata(), sstable.descriptor.version.correspondingMessagingVersion(), DeserializationHelper.Flag.LOCAL); SSTableSimpleIterator iterator = tombstoneOnly ? SSTableSimpleIterator.createTombstoneOnly(sstable.metadata(), dfile, sstable.header, helper, partitionLevelDeletion) diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java index 47de00c6a21a..e6998ee22348 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java @@ -21,7 +21,9 @@ import java.util.*; import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; +import com.google.common.collect.Sets; import org.apache.cassandra.db.streaming.CassandraOutgoingFile; import org.apache.cassandra.locator.InetAddressAndPort; @@ -90,12 +92,6 @@ protected Collection openSSTables(final Map openSSTables(final Map components = new HashSet<>(); - components.add(Component.DATA); - components.add(Component.PRIMARY_INDEX); - if (new File(desc.filenameFor(Component.SUMMARY)).exists()) - components.add(Component.SUMMARY); - if (new File(desc.filenameFor(Component.COMPRESSION_INFO)).exists()) - components.add(Component.COMPRESSION_INFO); - if (new File(desc.filenameFor(Component.STATS)).exists()) - components.add(Component.STATS); + Set components = mainComponentsPresent(desc); try { // To conserve memory, open SSTableReaders without bloom filters and discard // the index summary after calculating the file sections to stream and the estimated // number of keys for each endpoint. See CASSANDRA-5555 for details. - SSTableReader sstable = SSTableReader.openForBatch(desc, components, metadata); + SSTableReader sstable = desc.getFormat().getReaderFactory().openForBatch(desc, components, metadata); sstables.add(sstable); // calculate the sstable sections to stream as well as the estimated number of @@ -153,7 +141,7 @@ protected Collection openSSTables(final Map sstableSections = sstable.getPositionsForRanges(tokenRanges); long estimatedKeys = sstable.estimatedKeysForRanges(tokenRanges); - Ref ref = sstable.ref(); + Ref ref = sstable.ref(); OutgoingStream stream = new CassandraOutgoingFile(StreamOperation.BULK_LOAD, ref, sstableSections, tokenRanges, estimatedKeys); streamingDetails.put(endpoint, stream); } @@ -173,6 +161,20 @@ protected Collection openSSTables(final Map mainComponentsPresent(Descriptor desc) + { + Set lookFor = Sets.union(desc.getFormat().requiredComponents(), + ImmutableSet.of(Component.COMPRESSION_INFO)); + + Set components = new HashSet<>(); + for (Component component : lookFor) + { + if (new File(desc.filenameFor(component)).exists()) + components.add(component); + } + return components; + } + public StreamResultFuture stream() { return stream(Collections.emptySet()); diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java index 1be79abf8ba8..0dee13259e1f 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java @@ -43,7 +43,9 @@ public interface SSTableMultiWriter extends Transactional SSTableMultiWriter setOpenResult(boolean openResult); String getFilename(); - long getFilePointer(); + long getBytesWritten(); + long getOnDiskBytesWritten(); + int getSegmentCount(); TableId getTableId(); static void abortOrDie(SSTableMultiWriter writer) diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java index a3d5ae9a2bab..693735bdb144 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java @@ -27,7 +27,8 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.RowIndexEntry; +import org.apache.cassandra.io.sstable.format.RowIndexEntry; +import org.apache.cassandra.io.sstable.format.big.BigTableRowIndexEntry; import org.apache.cassandra.db.lifecycle.ILifecycleTransaction; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -63,13 +64,14 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme private final List preparedForCommit = new ArrayList<>(); private long currentlyOpenedEarlyAt; // the position (in MB) in the target file we last (re)opened at + private long bytesWritten; // the bytes written by previous writers, or zero if the current writer is the first writer private final List writers = new ArrayList<>(); private final boolean keepOriginals; // true if we do not want to obsolete the originals private final boolean eagerWriterMetaRelease; // true if the writer metadata should be released when switch is called private SSTableWriter writer; - private Map cachedKeys = new HashMap<>(); + private Map cachedKeys = new HashMap<>(); // for testing (TODO: remove when have byteman setup) private boolean throwEarly, throwLate; @@ -117,6 +119,11 @@ public SSTableWriter currentWriter() return writer; } + public long bytesWritten() + { + return bytesWritten + (writer == null ? 0 : writer.getFilePointer()); + } + public RowIndexEntry append(UnfilteredRowIterator partition) { // we do this before appending to ensure we can resetAndTruncate() safely if the append fails @@ -125,13 +132,13 @@ public RowIndexEntry append(UnfilteredRowIterator partition) RowIndexEntry index = writer.append(partition); if (DatabaseDescriptor.shouldMigrateKeycacheOnCompaction()) { - if (!transaction.isOffline() && index != null) + if (!transaction.isOffline() && index instanceof BigTableRowIndexEntry) { for (SSTableReader reader : transaction.originals()) { if (reader.getCachedPosition(key, false) != null) { - cachedKeys.put(key, index); + cachedKeys.put(key, (BigTableRowIndexEntry) index); break; } } @@ -169,14 +176,12 @@ private void maybeReopenEarly(DecoratedKey key) } else { - SSTableReader reader = writer.setMaxDataAge(maxAge).openEarly(); - if (reader != null) - { + writer.setMaxDataAge(maxAge).openEarly(reader -> { transaction.update(reader, false); currentlyOpenedEarlyAt = writer.getFilePointer(); moveStarts(reader, reader.last); transaction.checkpoint(); - } + }); } } } @@ -223,7 +228,7 @@ private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound) if (!cachedKeys.isEmpty()) { invalidateKeys = new ArrayList<>(cachedKeys.size()); - for (Map.Entry cacheKey : cachedKeys.entrySet()) + for (Map.Entry cacheKey : cachedKeys.entrySet()) { invalidateKeys.add(cacheKey.getKey()); newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue()); @@ -255,10 +260,13 @@ private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound) continue; } - DecoratedKey newStart = latest.firstKeyBeyond(lowerbound); - assert newStart != null; - SSTableReader replacement = latest.cloneWithNewStart(newStart, runOnClose); - transaction.update(replacement, true); + if (!transaction.isObsolete(latest)) + { + DecoratedKey newStart = latest.firstKeyBeyond(lowerbound); + assert newStart != null; + SSTableReader replacement = latest.cloneWithNewStart(newStart, runOnClose); + transaction.update(replacement, true); + } } } @@ -310,6 +318,8 @@ public void switchWriter(SSTableWriter newWriter) return; } + // Open fully completed sstables early. This is also required for the final sstable in a set (where newWriter + // is null) to permit the compilation of a canonical set of sstables (see View.select). if (preemptiveOpenInterval != Long.MAX_VALUE) { // we leave it as a tmp file, but we open it and add it to the Tracker @@ -320,6 +330,7 @@ public void switchWriter(SSTableWriter newWriter) } currentlyOpenedEarlyAt = 0; + bytesWritten += writer.getFilePointer(); writer = newWriter; } diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java index cfb1365649c7..213520dc022e 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java @@ -62,11 +62,6 @@ public String getFilename() return writer.getFilename(); } - public long getFilePointer() - { - return writer.getFilePointer(); - } - protected Throwable doCommit(Throwable accumulate) { return writer.commit(txn.commit(accumulate)); @@ -144,12 +139,12 @@ public static SSTableTxnWriter create(TableMetadataRef metadata, boolean isTransient, int sstableLevel, SerializationHeader header, - Collection indexes) + Collection indexGroups) { // if the column family store does not exist, we create a new default SSTableMultiWriter to use: LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.WRITE); MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(sstableLevel); - SSTableMultiWriter writer = SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, collector, header, indexes, txn); + SSTableMultiWriter writer = SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, collector, header, indexGroups, txn); return new SSTableTxnWriter(txn, writer); } diff --git a/src/java/org/apache/cassandra/io/sstable/ScannerList.java b/src/java/org/apache/cassandra/io/sstable/ScannerList.java new file mode 100644 index 000000000000..38ee3251769b --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/ScannerList.java @@ -0,0 +1,74 @@ +/* + * 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.sstable; + +import java.util.List; + +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; + +public class ScannerList implements AutoCloseable +{ + public final List scanners; + public ScannerList(List scanners) + { + this.scanners = scanners; + } + + public long getTotalBytesScanned() + { + long bytesScanned = 0L; + for (ISSTableScanner scanner : scanners) + bytesScanned += scanner.getBytesScanned(); + + return bytesScanned; + } + + public long getTotalCompressedSize() + { + long compressedSize = 0; + for (int i=0, isize=scanners.size(); i indexEntry = writer.append(partition); + RowIndexEntry indexEntry = writer.append(partition); return indexEntry != null; } @@ -75,11 +75,21 @@ public String getFilename() return writer.getFilename(); } - public long getFilePointer() + public long getBytesWritten() { return writer.getFilePointer(); } + public long getOnDiskBytesWritten() + { + return writer.getEstimatedOnDiskBytesWritten(); + } + + public int getSegmentCount() + { + return 1; + } + public TableId getTableId() { return writer.metadata().id; @@ -115,10 +125,10 @@ public static SSTableMultiWriter create(Descriptor descriptor, TableMetadataRef metadata, MetadataCollector metadataCollector, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { - SSTableWriter writer = SSTableWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, indexes, lifecycleNewTracker); + SSTableWriter writer = SSTableWriter.create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, indexGroups, lifecycleNewTracker); return new SimpleSSTableMultiWriter(writer, lifecycleNewTracker); } } diff --git a/src/java/org/apache/cassandra/io/sstable/UnsupportedSSTableException.java b/src/java/org/apache/cassandra/io/sstable/UnsupportedSSTableException.java new file mode 100644 index 000000000000..41e5ae04241e --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/UnsupportedSSTableException.java @@ -0,0 +1,29 @@ +/* + * 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.sstable; + +import java.io.File; + +public class UnsupportedSSTableException extends CorruptSSTableException +{ + public UnsupportedSSTableException(String msg, Throwable cause, File path) + { + super(msg, cause, path); + } +} diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/io/sstable/format/AbstractSSTableIterator.java similarity index 59% rename from src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java rename to src/java/org/apache/cassandra/io/sstable/format/AbstractSSTableIterator.java index fee45c232fe2..5d8f02811660 100644 --- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java +++ b/src/java/org/apache/cassandra/io/sstable/format/AbstractSSTableIterator.java @@ -15,26 +15,41 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.db.columniterator; +package org.apache.cassandra.io.sstable.format; + +import java.io.Closeable; import java.io.IOException; -import java.util.Comparator; import java.util.Iterator; import java.util.NoSuchElementException; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.db.*; -import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.rows.*; -import org.apache.cassandra.io.sstable.IndexInfo; -import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.db.BufferClusteringBound; +import org.apache.cassandra.db.ClusteringBound; +import org.apache.cassandra.db.Columns; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.UnfilteredDeserializer; +import org.apache.cassandra.db.UnfilteredValidation; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredSerializer; import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.rows.RangeTombstoneMarker; import org.apache.cassandra.io.util.FileDataInput; -import org.apache.cassandra.io.util.DataPosition; import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.ByteBufferUtil; -public abstract class AbstractSSTableIterator implements UnfilteredRowIterator +public abstract class AbstractSSTableIterator implements UnfilteredRowIterator { protected final SSTableReader sstable; // We could use sstable.metadata(), but that can change during execution so it's good hygiene to grab an immutable instance @@ -59,7 +74,7 @@ public abstract class AbstractSSTableIterator implements UnfilteredRowIterator protected AbstractSSTableIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, - RowIndexEntry indexEntry, + E indexEntry, Slices slices, ColumnFilter columnFilter, FileHandle ifile) @@ -157,10 +172,10 @@ private Slice nextSlice() */ protected abstract boolean hasMoreSlices(); - private static Row readStaticRow(SSTableReader sstable, - FileDataInput file, - DeserializationHelper helper, - Columns statics) throws IOException + public static Row readStaticRow(SSTableReader sstable, + FileDataInput file, + DeserializationHelper helper, + Columns statics) throws IOException { if (!sstable.header.hasStatic()) return Rows.EMPTY_STATIC_ROW; @@ -176,13 +191,13 @@ private static Row readStaticRow(SSTableReader sstable, } } - protected abstract Reader createReaderInternal(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile); + protected abstract Reader createReaderInternal(E indexEntry, FileDataInput file, boolean shouldCloseFile); - private Reader createReader(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile) + private Reader createReader(E indexEntry, FileDataInput file, boolean shouldCloseFile) { return slices.isEmpty() ? new NoRowsReader(file, shouldCloseFile) : createReaderInternal(indexEntry, file, shouldCloseFile); - }; + } public TableMetadata metadata() { @@ -255,7 +270,7 @@ private void slice(Slice slice) e.addSuppressed(suppressed); } sstable.markSuspect(); - throw new CorruptSSTableException(e, reader.file.getPath()); + throw new CorruptSSTableException(e, reader.toString()); } } @@ -286,24 +301,19 @@ public void close() catch (IOException e) { sstable.markSuspect(); - throw new CorruptSSTableException(e, reader.file.getPath()); + throw new CorruptSSTableException(e, reader.toString()); } } - protected abstract class Reader implements Iterator - { - private final boolean shouldCloseFile; - public FileDataInput file; - - protected UnfilteredDeserializer deserializer; + public abstract class RowReader extends Reader { + public UnfilteredDeserializer deserializer; // Records the currently open range tombstone (if any) - protected DeletionTime openMarker = null; + public DeletionTime openMarker; - protected Reader(FileDataInput file, boolean shouldCloseFile) + protected RowReader(FileDataInput file, boolean shouldCloseFile) { - this.file = file; - this.shouldCloseFile = shouldCloseFile; + super(file, shouldCloseFile); if (file != null) createDeserializer(); @@ -315,7 +325,7 @@ private void createDeserializer() deserializer = UnfilteredDeserializer.create(metadata, file, sstable.header, helper); } - protected void seekToPosition(long position) throws IOException + public void seekToPosition(long position) throws IOException { // This may be the first time we're actually looking into the file if (file == null) @@ -326,6 +336,7 @@ protected void seekToPosition(long position) throws IOException else { file.seek(position); + deserializer.clearState(); } } @@ -334,6 +345,151 @@ protected void updateOpenMarker(RangeTombstoneMarker marker) // Note that we always read index blocks in forward order so this method is always called in forward order openMarker = marker.isOpen(false) ? marker.openDeletionTime(false) : null; } + } + + protected class ForwardReader extends RowReader + { + // The start of the current slice. This will be null as soon as we know we've passed that bound. + protected ClusteringBound start; + // The end of the current slice. Will never be null. + protected ClusteringBound end = BufferClusteringBound.TOP; + + protected Unfiltered next; // the next element to return: this is computed by hasNextInternal(). + + protected boolean sliceDone; // set to true once we know we have no more result for the slice. This is in particular + // used by the indexed reader when we know we can't have results based on the index. + + public ForwardReader(FileDataInput file, boolean shouldCloseFile) + { + super(file, shouldCloseFile); + } + + public void setForSlice(Slice slice) throws IOException + { + start = slice.start().isBottom() ? null : slice.start(); + end = slice.end(); + + sliceDone = false; + next = null; + } + + // Skip all data that comes before the currently set slice. + // Return what should be returned at the end of this, or null if nothing should. + private Unfiltered handlePreSliceData() throws IOException + { + assert deserializer != null; + + // Note that the following comparison is not strict. The reason is that the only cases + // where it can be == is if the "next" is a RT start marker (either a '[' of a ')[' boundary), + // and if we had a strict inequality and an open RT marker before this, we would issue + // the open marker first, and then return then next later, which would send in the + // stream both '[' (or '(') and then ')[' for the same clustering value, which is wrong. + // By using a non-strict inequality, we avoid that problem (if we do get ')[' for the same + // clustering value than the slice, we'll simply record it in 'openMarker'). + while (deserializer.hasNext() && deserializer.compareNextTo(start) <= 0) + { + if (deserializer.nextIsRow()) + deserializer.skipNext(); + else + updateOpenMarker((RangeTombstoneMarker)deserializer.readNext()); + } + + ClusteringBound sliceStart = start; + start = null; + + // We've reached the beginning of our queried slice. If we have an open marker + // we should return that first. + if (openMarker != null) + return new RangeTombstoneBoundMarker(sliceStart, openMarker); + + return null; + } + + // Compute the next element to return, assuming we're in the middle to the slice + // and the next element is either in the slice, or just after it. Returns null + // if we're done with the slice. + protected Unfiltered computeNext() throws IOException + { + assert deserializer != null; + + while (true) + { + // We use a same reasoning as in handlePreSliceData regarding the strictness of the inequality below. + // We want to exclude deserialized unfiltered equal to end, because 1) we won't miss any rows since those + // woudn't be equal to a slice bound and 2) a end bound can be equal to a start bound + // (EXCL_END(x) == INCL_START(x) for instance) and in that case we don't want to return start bound because + // it's fundamentally excluded. And if the bound is a end (for a range tombstone), it means it's exactly + // our slice end, but in that case we will properly close the range tombstone anyway as part of our "close + // an open marker" code in hasNextInterna + if (!deserializer.hasNext() || deserializer.compareNextTo(end) >= 0) + return null; + + Unfiltered next = deserializer.readNext(); + UnfilteredValidation.maybeValidateUnfiltered(next, metadata(), key, sstable); + // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne. + if (next.isEmpty()) + continue; + + if (next.kind() == Unfiltered.Kind.RANGE_TOMBSTONE_MARKER) + updateOpenMarker((RangeTombstoneMarker) next); + return next; + } + } + + protected boolean hasNextInternal() throws IOException + { + if (next != null) + return true; + + if (sliceDone) + return false; + + if (start != null) + { + Unfiltered unfiltered = handlePreSliceData(); + if (unfiltered != null) + { + next = unfiltered; + return true; + } + } + + next = computeNext(); + if (next != null) + return true; + + // for current slice, no data read from deserialization + sliceDone = true; + // If we have an open marker, we should not close it, there could be more slices + if (openMarker != null) + { + next = new RangeTombstoneBoundMarker(end, openMarker); + return true; + } + return false; + } + + protected Unfiltered nextInternal() throws IOException + { + if (!hasNextInternal()) + throw new NoSuchElementException(); + + Unfiltered toReturn = next; + next = null; + return toReturn; + } + } + + protected abstract class Reader implements Iterator, Closeable + { + public FileDataInput file; + protected final boolean shouldCloseFile; + + protected Reader(FileDataInput file, boolean shouldCloseFile) + { + this.file = file; + this.shouldCloseFile = shouldCloseFile; + } public boolean hasNext() { @@ -352,7 +508,7 @@ public boolean hasNext() e.addSuppressed(suppressed); } sstable.markSuspect(); - throw new CorruptSSTableException(e, reader.file.getPath()); + throw new CorruptSSTableException(e, toString()); } } @@ -373,7 +529,7 @@ public Unfiltered next() e.addSuppressed(suppressed); } sstable.markSuspect(); - throw new CorruptSSTableException(e, reader.file.getPath()); + throw new CorruptSSTableException(e, toString()); } } @@ -381,26 +537,34 @@ public Unfiltered next() public abstract void setForSlice(Slice slice) throws IOException; protected abstract boolean hasNextInternal() throws IOException; + protected abstract Unfiltered nextInternal() throws IOException; + @Override public void close() throws IOException { if (shouldCloseFile && file != null) file.close(); } + + @Override + public String toString() + { + return file != null ? file.getPath() : "null"; + } } // Reader for when we have Slices.NONE but need to read static row or partition level deletion - private class NoRowsReader extends AbstractSSTableIterator.Reader + private class NoRowsReader extends Reader { - private NoRowsReader(FileDataInput file, boolean shouldCloseFile) + public NoRowsReader(FileDataInput file, boolean shouldCloseFile) { super(file, shouldCloseFile); } public void setForSlice(Slice slice) throws IOException { - return; + // no-op } protected boolean hasNextInternal() throws IOException @@ -413,199 +577,4 @@ protected Unfiltered nextInternal() throws IOException throw new NoSuchElementException(); } } - - // Used by indexed readers to store where they are of the index. - public static class IndexState implements AutoCloseable - { - private final Reader reader; - private final ClusteringComparator comparator; - - private final RowIndexEntry indexEntry; - private final RowIndexEntry.IndexInfoRetriever indexInfoRetriever; - private final boolean reversed; - - private int currentIndexIdx; - - // Marks the beginning of the block corresponding to currentIndexIdx. - private DataPosition mark; - - public IndexState(Reader reader, ClusteringComparator comparator, RowIndexEntry indexEntry, boolean reversed, FileHandle indexFile) - { - this.reader = reader; - this.comparator = comparator; - this.indexEntry = indexEntry; - this.indexInfoRetriever = indexEntry.openWithIndex(indexFile); - this.reversed = reversed; - this.currentIndexIdx = reversed ? indexEntry.columnsIndexCount() : -1; - } - - public boolean isDone() - { - return reversed ? currentIndexIdx < 0 : currentIndexIdx >= indexEntry.columnsIndexCount(); - } - - // Sets the reader to the beginning of blockIdx. - public void setToBlock(int blockIdx) throws IOException - { - if (blockIdx >= 0 && blockIdx < indexEntry.columnsIndexCount()) - { - reader.seekToPosition(columnOffset(blockIdx)); - mark = reader.file.mark(); - reader.deserializer.clearState(); - } - - currentIndexIdx = blockIdx; - reader.openMarker = blockIdx > 0 ? index(blockIdx - 1).endOpenMarker : null; - } - - private long columnOffset(int i) throws IOException - { - return indexEntry.position + index(i).offset; - } - - public int blocksCount() - { - return indexEntry.columnsIndexCount(); - } - - // Update the block idx based on the current reader position if we're past the current block. - // This only makes sense for forward iteration (for reverse ones, when we reach the end of a block we - // should seek to the previous one, not update the index state and continue). - public void updateBlock() throws IOException - { - assert !reversed; - - // If we get here with currentBlockIdx < 0, it means setToBlock() has never been called, so it means - // we're about to read from the beginning of the partition, but haven't "prepared" the IndexState yet. - // Do so by setting us on the first block. - if (currentIndexIdx < 0) - { - setToBlock(0); - return; - } - - while (currentIndexIdx + 1 < indexEntry.columnsIndexCount() && isPastCurrentBlock()) - { - reader.openMarker = currentIndex().endOpenMarker; - ++currentIndexIdx; - - // We have to set the mark, and we have to set it at the beginning of the block. So if we're not at the beginning of the block, this forces us to a weird seek dance. - // This can only happen when reading old file however. - long startOfBlock = columnOffset(currentIndexIdx); - long currentFilePointer = reader.file.getFilePointer(); - if (startOfBlock == currentFilePointer) - { - mark = reader.file.mark(); - } - else - { - reader.seekToPosition(startOfBlock); - mark = reader.file.mark(); - reader.seekToPosition(currentFilePointer); - } - } - } - - // Check if we've crossed an index boundary (based on the mark on the beginning of the index block). - public boolean isPastCurrentBlock() throws IOException - { - assert reader.deserializer != null; - return reader.file.bytesPastMark(mark) >= currentIndex().width; - } - - public int currentBlockIdx() - { - return currentIndexIdx; - } - - public IndexInfo currentIndex() throws IOException - { - return index(currentIndexIdx); - } - - public IndexInfo index(int i) throws IOException - { - return indexInfoRetriever.columnsIndex(i); - } - - // Finds the index of the first block containing the provided bound, starting at the provided index. - // Will be -1 if the bound is before any block, and blocksCount() if it is after every block. - public int findBlockIndex(ClusteringBound bound, int fromIdx) throws IOException - { - if (bound.isBottom()) - return -1; - if (bound.isTop()) - return blocksCount(); - - return indexFor(bound, fromIdx); - } - - public int indexFor(ClusteringPrefix name, int lastIndex) throws IOException - { - IndexInfo target = new IndexInfo(name, name, 0, 0, null); - /* - Take the example from the unit test, and say your index looks like this: - [0..5][10..15][20..25] - and you look for the slice [13..17]. - - When doing forward slice, we are doing a binary search comparing 13 (the start of the query) - to the lastName part of the index slot. You'll end up with the "first" slot, going from left to right, - that may contain the start. - - When doing a reverse slice, we do the same thing, only using as a start column the end of the query, - i.e. 17 in this example, compared to the firstName part of the index slots. bsearch will give us the - first slot where firstName > start ([20..25] here), so we subtract an extra one to get the slot just before. - */ - int startIdx = 0; - int endIdx = indexEntry.columnsIndexCount() - 1; - - if (reversed) - { - if (lastIndex < endIdx) - { - endIdx = lastIndex; - } - } - else - { - if (lastIndex > 0) - { - startIdx = lastIndex; - } - } - - int index = binarySearch(target, comparator.indexComparator(reversed), startIdx, endIdx); - return (index < 0 ? -index - (reversed ? 2 : 1) : index); - } - - private int binarySearch(IndexInfo key, Comparator c, int low, int high) throws IOException - { - while (low <= high) - { - int mid = (low + high) >>> 1; - IndexInfo midVal = index(mid); - int cmp = c.compare(midVal, key); - - if (cmp < 0) - low = mid + 1; - else if (cmp > 0) - high = mid - 1; - else - return mid; - } - return -(low + 1); - } - - @Override - public String toString() - { - return String.format("IndexState(indexSize=%d, currentBlock=%d, reversed=%b)", indexEntry.columnsIndexCount(), currentIndexIdx, reversed); - } - - @Override - public void close() throws IOException - { - indexInfoRetriever.close(); - } - } } diff --git a/src/java/org/apache/cassandra/io/sstable/format/PartitionIndexIterator.java b/src/java/org/apache/cassandra/io/sstable/format/PartitionIndexIterator.java new file mode 100644 index 000000000000..ff8e7c31f7fa --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/format/PartitionIndexIterator.java @@ -0,0 +1,82 @@ +/* + * 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.sstable.format; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Iterator over the partitions of an sstable. + *

    + * The index iterator starts with a key/position ready. {@link #advance()} should be used to move to the next key; + * iteration completes when {@link #advance()} returns {@code false}. For testing the state of iterator any time + * {@link #isExhausted()} can be used. + */ +public interface PartitionIndexIterator extends Closeable +{ + /** + * Current key + */ + public ByteBuffer key(); + + /** + * Position in the component preferred for reading keys. This is specific to SSTable implementation + */ + long keyPosition(); + + /** + * Position in the data file where the associated content resides + */ + public long dataPosition(); + + /** + * Moves the iterator forward. Returns false if we reach EOF and there nothing more to read + */ + public boolean advance() throws IOException; + + /** + * Closes the iterator quietly + */ + public void close(); + + /** + * Returns true if we reach EOF + */ + boolean isExhausted(); + + /** + * Returns the current position in index file + */ + long indexPosition(); + + /** + * Sets the current position in index file + */ + void indexPosition(long position) throws IOException; + + /** + * Returns length of the index file + */ + long indexLength(); + + /** + * Resets the iterator to the initial position + */ + void reset() throws IOException; +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java index ef4deb719331..3720b86a592d 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java @@ -67,7 +67,7 @@ public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long r this.format = format; this.lifecycleNewTracker = lifecycleNewTracker; this.header = header; - boundaries = db.positions; + boundaries = db.getPositions(); if (boundaries == null) { Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize); @@ -115,7 +115,7 @@ public Collection finish(long repairedAt, long maxDataAge, boolea currentWriter = null; for (SSTableMultiWriter writer : finishedWriters) { - if (writer.getFilePointer() > 0) + if (writer.getBytesWritten() > 0) finishedReaders.addAll(writer.finish(repairedAt, maxDataAge, openResult)); else SSTableMultiWriter.abortOrDie(writer); @@ -131,7 +131,7 @@ public Collection finish(boolean openResult) currentWriter = null; for (SSTableMultiWriter writer : finishedWriters) { - if (writer.getFilePointer() > 0) + if (writer.getBytesWritten() > 0) finishedReaders.addAll(writer.finish(openResult)); else SSTableMultiWriter.abortOrDie(writer); @@ -159,9 +159,27 @@ public String getFilename() } @Override - public long getFilePointer() + public long getBytesWritten() { - return currentWriter.getFilePointer(); + long bytesWritten = currentWriter.getBytesWritten(); + for (SSTableMultiWriter writer : finishedWriters) + bytesWritten += writer.getBytesWritten(); + return bytesWritten; + } + + @Override + public long getOnDiskBytesWritten() + { + long bytesWritten = currentWriter.getOnDiskBytesWritten(); + for (SSTableMultiWriter writer : finishedWriters) + bytesWritten += writer.getOnDiskBytesWritten(); + return bytesWritten; + } + + @Override + public int getSegmentCount() + { + return finishedWriters.size() + 1; } @Override diff --git a/src/java/org/apache/cassandra/io/sstable/format/RowIndexEntry.java b/src/java/org/apache/cassandra/io/sstable/format/RowIndexEntry.java new file mode 100644 index 000000000000..80d03b2ff201 --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/format/RowIndexEntry.java @@ -0,0 +1,66 @@ +/* + * 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.sstable.format; + +import org.apache.cassandra.cache.IMeasurableMemory; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.utils.ObjectSizes; + +/** + * The base RowIndexEntry is not stored on disk, only specifies a position in the data file + */ +public class RowIndexEntry implements IMeasurableMemory +{ + private static final long EMPTY_SIZE = ObjectSizes.measure(new RowIndexEntry(0)); + + /** + * Row position in a data file + */ + public final long position; + + public RowIndexEntry(long position) + { + this.position = position; + } + + /** + * @return true if this index entry contains the row-level tombstone and column summary. Otherwise, + * caller should fetch these from the row header. + */ + public boolean isIndexed() + { + return columnsIndexCount() > 1; + } + + public DeletionTime deletionTime() + { + throw new UnsupportedOperationException(); + } + + public int columnsIndexCount() + { + return 0; + } + + @Override + public long unsharedHeapSize() + { + return EMPTY_SIZE; + } +} diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java index f0b6bac86144..0cbc57003b84 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableFlushObserver.java @@ -18,6 +18,9 @@ package org.apache.cassandra.io.sstable.format; import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.Unfiltered; /** @@ -35,21 +38,75 @@ public interface SSTableFlushObserver * but before any cells are processed (see {@link #nextUnfilteredCluster(Unfiltered)}). * * @param key The key being appended to SSTable. - * @param indexPosition The position of the key in the SSTable PRIMARY_INDEX file. + * @param position The position of the key in the component preferred for reading keys */ - void startPartition(DecoratedKey key, long indexPosition); + void startPartition(DecoratedKey key, long position); /** - * Called after the unfiltered cluster is written to the sstable. - * Will be preceded by a call to {@code startPartition(DecoratedKey, long)}, - * and the cluster should be assumed to belong to that partition. + * Called when the deletion time of a partition is written to the sstable. * - * @param unfilteredCluster The unfiltered cluster being added to SSTable. + * Will be preceded by a call to {@link #startPartition(DecoratedKey, long)}, + * and the deletion time should be assumed to belong to that partition. + * + * @param deletionTime the partition-level deletion time being written to the SSTable + * @param position the position of the written deletion time in the data file, + * as required by {@link SSTableReader#partitionLevelDeletionAt(long)} + */ + void partitionLevelDeletion(DeletionTime deletionTime, long position); + + /** + * Called when the static row of a partition is written to the sstable. + * + * Will be preceded by a call to {@link #startPartition(DecoratedKey, long)}, + * and the static row should be assumed to belong to that partition. + * + * @param staticRow the static row being written to the SSTable + * @param position the position of the written static row in the data file, + * as required by {@link SSTableReader#staticRowAt(long, ColumnFilter)} */ - void nextUnfilteredCluster(Unfiltered unfilteredCluster); + void staticRow(Row staticRow, long position); + + /** + * Called after an unfiltered is written to the sstable. + * + * Will be preceded by a call to {@link #startPartition(DecoratedKey, long)}, + * and the unfiltered should be assumed to belong to that partition. + * + * Implementations overriding {@link #nextUnfilteredCluster(Unfiltered, long)} shouldn't implement this method + * since only one of the two methods is required. + * + * @param unfiltered the unfiltered being written to the SSTable + */ + default void nextUnfilteredCluster(Unfiltered unfiltered) + { + } + + /** + * Called after an unfiltered is written to the sstable. + * + * Will be preceded by a call to {@link #startPartition(DecoratedKey, long)}, + * and the unfiltered should be assumed to belong to that partition. + * + * Implementations overriding {@link #nextUnfilteredCluster(Unfiltered)} shouldn't implement this method + * since only one of the two methods is required. + * + * @param unfiltered the unfiltered being written to the SSTable + * @param position the position of the written unfiltered in the data file, + * as required by {@link SSTableReader#clusteringAt(long)} + * and {@link SSTableReader#unfilteredAt(long, ColumnFilter)} + */ + default void nextUnfilteredCluster(Unfiltered unfiltered, long position) + { + nextUnfilteredCluster(unfiltered); + } /** * Called when all data is written to the file and it's ready to be finished up. */ void complete(); + + /** + * Clean up resources on error. There should be no side effects if called multiple times. + */ + default void abort(Throwable accumulator) {} } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java index 14f660258fbe..53d2d16d0cc2 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java @@ -17,20 +17,22 @@ */ package org.apache.cassandra.io.sstable.format; +import java.util.Set; + import com.google.common.base.CharMatcher; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.db.RowIndexEntry; -import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.big.BigFormat; +import org.apache.cassandra.io.sstable.format.trieindex.TrieIndexFormat; /** * Provides the accessors to data on disk. */ public interface SSTableFormat { - static boolean enableSSTableDevelopmentTestMode = Boolean.getBoolean("cassandra.test.sstableformatdevelopment"); + public final static String FORMAT_DEFAULT_PROP = "cassandra.sstable.format.default"; + Type getType(); Version getLatestVersion(); Version getVersion(String version); @@ -38,22 +40,23 @@ public interface SSTableFormat SSTableWriter.Factory getWriterFactory(); SSTableReader.Factory getReaderFactory(); - RowIndexEntry.IndexSerializer getIndexSerializer(TableMetadata metadata, Version version, SerializationHeader header); - - public static enum Type + public enum Type { //The original sstable format - BIG("big", BigFormat.instance); + BIG("big", BigFormat.instance), + + //Sstable format with trie indices + BTI("bti", TrieIndexFormat.instance); public final SSTableFormat info; public final String name; public static Type current() { - return BIG; + return Type.valueOf(System.getProperty(FORMAT_DEFAULT_PROP, BTI.name()).toUpperCase()); } - private Type(String name, SSTableFormat info) + Type(String name, SSTableFormat info) { //Since format comes right after generation //we disallow formats with numeric names @@ -74,4 +77,27 @@ public static Type validate(String name) throw new IllegalArgumentException("No Type constant " + name); } } + + /** + * Returns components required by the particular implementation of SSTable reader so that it can operate on + * the SSTable in a regular way. + */ + Set requiredComponents(); + + /** + * Returns all the components, both mandatory and optional, which are used by the particular implemetation of + * SSTable format. + */ + Set supportedComponents(); + + /** + * Returns all the components of the particular implementation of SSTable format which are suitable for streaming. + */ + Set streamingComponents(); + + /** + * Returns all primary index components required for index iteration and reading keys + */ + Set primaryIndexComponents(); + } 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 ea40f34f265b..04f93eb20abd 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -17,17 +17,41 @@ */ package org.apache.cassandra.io.sstable.format; -import java.io.*; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; import java.lang.ref.WeakReference; import java.nio.ByteBuffer; -import java.util.*; -import java.util.concurrent.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collector; + +import javax.annotation.Nonnull; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.google.common.primitives.Longs; import com.google.common.util.concurrent.RateLimiter; import org.slf4j.Logger; @@ -36,18 +60,34 @@ 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.InstrumentingCache; import org.apache.cassandra.cache.KeyCacheKey; import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor; import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.*; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringBoundOrBoundary; +import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.db.compaction.Scrubber; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.rows.BTreeRow; import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.DeserializationHelper; import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.Unfiltered; import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredSerializer; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Range; @@ -57,22 +97,52 @@ import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.compress.CompressionMetadata; -import org.apache.cassandra.io.sstable.*; -import org.apache.cassandra.io.sstable.metadata.*; -import org.apache.cassandra.io.util.*; +import org.apache.cassandra.io.sstable.BloomFilterTracker; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.Downsampling; +import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.IndexSummary; +import org.apache.cassandra.io.sstable.IndexSummaryBuilder; +import org.apache.cassandra.io.sstable.KeyIterator; +import org.apache.cassandra.io.sstable.SSTable; +import org.apache.cassandra.io.sstable.format.big.BigTableRowIndexEntry; +import org.apache.cassandra.io.sstable.metadata.CompactionMetadata; +import org.apache.cassandra.io.sstable.metadata.MetadataComponent; +import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.apache.cassandra.io.sstable.metadata.ValidationMetadata; +import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; +import org.apache.cassandra.io.util.ChannelProxy; +import org.apache.cassandra.io.util.CheckedFunction; +import org.apache.cassandra.io.util.DataOutputStreamPlus; +import org.apache.cassandra.io.util.FileDataInput; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.io.util.RandomAccessReader; import org.apache.cassandra.metrics.RestorableMeter; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.service.CacheService; -import org.apache.cassandra.utils.*; +import org.apache.cassandra.utils.BloomFilter; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.EstimatedHistogram; +import org.apache.cassandra.utils.ExecutorUtils; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.IFilter; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.NativeLibrary; +import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.concurrent.Ref; +import org.apache.cassandra.utils.concurrent.RefCounted; import org.apache.cassandra.utils.concurrent.SelfRefCounted; -import org.apache.cassandra.utils.BloomFilterSerializer; import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR; @@ -157,22 +227,18 @@ private static ScheduledThreadPoolExecutor initSyncExecutor() public static final Comparator maxTimestampDescending = (o1, o2) -> Long.compare(o2.getMaxTimestamp(), o1.getMaxTimestamp()); public static final Comparator maxTimestampAscending = (o1, o2) -> Long.compare(o1.getMaxTimestamp(), o2.getMaxTimestamp()); + public abstract boolean hasIndex(); + // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition public static final class UniqueIdentifier {} - public static final Comparator sstableComparator = (o1, o2) -> o1.first.compareTo(o2.first); + public static final Comparator firstKeyComparator = (o1, o2) -> o1.getFirst().compareTo(o2.getFirst()); public static final Comparator generationReverseComparator = (o1, o2) -> -Integer.compare(o1.descriptor.generation, o2.descriptor.generation); - public static final Ordering sstableOrdering = Ordering.from(sstableComparator); + public static final Ordering firstKeyOrdering = Ordering.from(firstKeyComparator); - public static final Comparator sizeComparator = new Comparator() - { - public int compare(SSTableReader o1, SSTableReader o2) - { - return Longs.compare(o1.onDiskLength(), o2.onDiskLength()); - } - }; + public static final Comparator sizeComparator = (o1, o2) -> Longs.compare(o1.onDiskLength(), o2.onDiskLength()); /** * maxDataAge is a timestamp in local server time (e.g. System.currentTimeMilli) which represents an upper bound @@ -205,11 +271,9 @@ public enum OpenReason protected final IFilter bf; public final IndexSummary indexSummary; - protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer; + protected InstrumentingCache keyCache; - protected InstrumentingCache keyCache; - - protected final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker(); + private volatile BloomFilterTracker bloomFilterTracker = BloomFilterTracker.createNoopTracker(); // technically isCompacted is not necessary since it should never be unreferenced unless it is also compacted, // but it seems like a good extra layer of protection against reference counting bugs to not delete data based on that alone @@ -223,13 +287,23 @@ public enum OpenReason protected final AtomicLong keyCacheHit = new AtomicLong(0); protected final AtomicLong keyCacheRequest = new AtomicLong(0); - private final InstanceTidier tidy; + protected final InstanceTidier tidy; private final Ref selfRef; private RestorableMeter readMeter; private volatile double crcCheckChance; + public static Iterable selectOnlyBigTableReaders(Iterable readers) + { + return Iterables.filter(readers, tr -> tr.descriptor.formatType == SSTableFormat.Type.BIG); + } + + public static T selectOnlyBigTableReaders(Collection readers, Collector collector) + { + return readers.stream().filter(tr -> tr.descriptor.formatType == SSTableFormat.Type.BIG).collect(collector); + } + /** * Calculate approximate key count. * If cardinality estimator is available on all given sstables, then this method use them to estimate @@ -239,7 +313,7 @@ public enum OpenReason * @param sstables SSTables to calculate key count * @return estimated key count */ - public static long getApproximateKeyCount(Iterable sstables) + public static long getApproximateKeyCount(Iterable sstables) { long count = -1; @@ -296,6 +370,31 @@ public static long getApproximateKeyCount(Iterable sstables) return count; } + /** + * The key cardinality estimator for the sstable, if it can be loaded. + * + * @return the sstable key cardinality estimator created during flush/compaction, or {@code null} if that estimator + * cannot be loaded for any reason. + */ + @VisibleForTesting + public ICardinality keyCardinalityEstimator() + { + if (openReason == OpenReason.EARLY) + return null; + + try + { + CompactionMetadata metadata = (CompactionMetadata) descriptor.getMetadataSerializer() + .deserialize(descriptor, MetadataType.COMPACTION); + return metadata == null ? null : metadata.cardinalityEstimator; + } + catch (IOException e) + { + logger.warn("Reading cardinality from Statistics.db failed for {}.", this, e); + return null; + } + } + /** * Estimates how much of the keys we would keep if the sstables were compacted together */ @@ -362,24 +461,24 @@ public static SSTableReader open(Descriptor descriptor) return open(descriptor, metadata); } - public static SSTableReader open(Descriptor desc, TableMetadataRef metadata) + private static SSTableReader open(Descriptor desc, TableMetadataRef metadata) { return open(desc, componentsFor(desc), metadata); } - public static SSTableReader open(Descriptor descriptor, Set components, TableMetadataRef metadata) + private static SSTableReader open(Descriptor descriptor, Set components, TableMetadataRef metadata) { return open(descriptor, components, metadata, true, false); } // use only for offline or "Standalone" operations - public static SSTableReader openNoValidation(Descriptor descriptor, Set components, ColumnFamilyStore cfs) + private static SSTableReader openNoValidation(Descriptor descriptor, Set components, ColumnFamilyStore cfs) { return open(descriptor, components, cfs.metadata, false, true); } // use only for offline or "Standalone" operations - public static SSTableReader openNoValidation(Descriptor descriptor, TableMetadataRef metadata) + private static SSTableReader openNoValidation(Descriptor descriptor, TableMetadataRef metadata) { return open(descriptor, componentsFor(descriptor), metadata, false, true); } @@ -393,7 +492,7 @@ public static SSTableReader openNoValidation(Descriptor descriptor, TableMetadat * @return opened SSTableReader * @throws IOException */ - public static SSTableReader openForBatch(Descriptor descriptor, Set components, TableMetadataRef metadata) + private static SSTableReader openForBatch(Descriptor descriptor, Set components, TableMetadataRef metadata) { // Minimum components without which we can't do anything assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor; @@ -447,11 +546,12 @@ public static SSTableReader openForBatch(Descriptor descriptor, Set c * @return {@link SSTableReader} * @throws IOException */ + @VisibleForTesting public static SSTableReader open(Descriptor descriptor, - Set components, - TableMetadataRef metadata, - boolean validate, - boolean isOffline) + Set components, + TableMetadataRef metadata, + boolean validate, + boolean isOffline) { // Minimum components without which we can't do anything assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor; @@ -537,7 +637,7 @@ public void run() SSTableReader sstable; try { - sstable = open(entry.getKey(), entry.getValue(), metadata); + sstable = entry.getKey().getFormat().getReaderFactory().open(entry.getKey(), entry.getValue(), metadata); } catch (CorruptSSTableException ex) { @@ -571,6 +671,21 @@ public void run() } + + /** + * Set the Bloom Filter tracker. The argument supplied is obtained + * from the the property of the owning CFS. + **/ + public void setBloomFilterTracker(BloomFilterTracker bloomFilterTracker) + { + this.bloomFilterTracker = bloomFilterTracker; + } + + protected BloomFilterTracker getBloomFilterTracker() + { + return this.bloomFilterTracker; + } + /** * Open a RowIndexedReader which already has its state initialized (by SSTableWriter). */ @@ -659,7 +774,6 @@ protected SSTableReader(final Descriptor desc, this.bf = bf; this.maxDataAge = maxDataAge; this.openReason = openReason; - this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata.get(), desc.version, header); tidy = new InstanceTidier(descriptor, metadata.id); selfRef = new Ref<>(this, tidy); } @@ -681,6 +795,16 @@ public static long getTotalUncompressedBytes(Iterable sstables) return sum; } + public abstract PartitionIndexIterator allKeysIterator() throws IOException; + + /** + * Partition iterator used only for scrubing (see {@link Scrubber} and {@link ScrubPartitionIterator}). + * + * @return iterator for scrubing or {@code null} if this {@link SSTableReader} doesn't have the iterator + * implemenation (this may be the case if there is no index file for the iterator) + */ + public abstract ScrubPartitionIterator scrubPartitionsIterator() throws IOException; + public boolean equals(Object that) { return that instanceof SSTableReader && ((SSTableReader) that).descriptor.equals(this.descriptor); @@ -697,17 +821,25 @@ public String getFilename() } public void setupOnline() + { + final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata().id); + setupOnline(cfs); + } + + public void setupOnline(ColumnFamilyStore cfs) { // under normal operation we can do this at any time, but SSTR is also used outside C* proper, // e.g. by BulkLoader, which does not initialize the cache. As a kludge, we set up the cache // here when we know we're being wired into the rest of the server infrastructure. - InstrumentingCache maybeKeyCache = CacheService.instance.keyCache; + InstrumentingCache maybeKeyCache = CacheService.instance.keyCache; if (maybeKeyCache.getCapacity() > 0) keyCache = maybeKeyCache; - final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata().id); if (cfs != null) + { setCrcCheckChance(cfs.getCrcCheckChance()); + setBloomFilterTracker(cfs.getBloomFilterTracker()); + } } /** @@ -740,7 +872,7 @@ public static void saveBloomFilter(Descriptor descriptor, IFilter filter) File filterFile = new File(descriptor.filenameFor(Component.FILTER)); try (DataOutputStreamPlus stream = new BufferedDataOutputStreamPlus(new FileOutputStream(filterFile))) { - BloomFilterSerializer.serialize((BloomFilter) filter, stream); + BloomFilter.serializer.serialize((BloomFilter) filter, stream); stream.flush(); } catch (IOException e) @@ -759,7 +891,7 @@ public static void saveBloomFilter(Descriptor descriptor, IFilter filter) * * @param task to be guarded by sstable lock */ - public R runWithLock(CheckedFunction task) throws IOException + public R runWithLock(CheckedFunction task) throws E { synchronized (tidy.global) { @@ -917,7 +1049,7 @@ public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable run } } - private static class DropPageCache implements Runnable + protected static class DropPageCache implements Runnable { final FileHandle dfile; final long dfilePosition; @@ -925,7 +1057,7 @@ private static class DropPageCache implements Runnable final long ifilePosition; final Runnable andThen; - private DropPageCache(FileHandle dfile, long dfilePosition, FileHandle ifile, long ifilePosition, Runnable andThen) + public DropPageCache(FileHandle dfile, long dfilePosition, FileHandle ifile, long ifilePosition, Runnable andThen) { this.dfile = dfile; this.dfilePosition = dfilePosition; @@ -995,25 +1127,16 @@ else if (samplingLevel < indexSummary.getSamplingLevel()) private IndexSummary buildSummaryAtLevel(int newSamplingLevel) throws IOException { // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary. - RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))); - try + try (PartitionIndexIterator iterator = allKeysIterator(); + IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata().params.minIndexInterval, newSamplingLevel)) { - long indexSize = primaryIndex.length(); - try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata().params.minIndexInterval, newSamplingLevel)) + while (!iterator.isExhausted()) { - long indexPosition; - while ((indexPosition = primaryIndex.getFilePointer()) != indexSize) - { - summaryBuilder.maybeAddEntry(decorateKey(ByteBufferUtil.readWithShortLength(primaryIndex)), indexPosition); - RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version); - } - - return summaryBuilder.build(getPartitioner()); + summaryBuilder.maybeAddEntry(decorateKey(iterator.key()), iterator.keyPosition()); + iterator.advance(); } - } - finally - { - FileUtils.closeQuietly(primaryIndex); + + return summaryBuilder.build(getPartitioner()); } } @@ -1022,6 +1145,21 @@ public RestorableMeter getReadMeter() return readMeter; } + /** + * Called by {@link org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy} and other compaction + * strategies to determine the read hotness of this sstables, this method returna a "read hotness" which is + * calculated by looking at the last two hours read rate and dividing this number by the estimated number of keys. + *

    + * Note that some system tables do not have read meters, in which case this method will return zero. + * + * @return the last two hours read rate per estimated key + */ + public double hotness() + { + // system tables don't have read meters, just use 0.0 for the hotness + return readMeter == null ? 0.0 : readMeter.twoHourRate() / estimatedKeys(); + } + public int getIndexSummarySamplingLevel() { return indexSummary.getSamplingLevel(); @@ -1044,10 +1182,11 @@ public double getEffectiveIndexInterval() public void releaseSummary() { - tidy.releaseSummary(); + indexSummary.close(); + assert indexSummary.isCleanedUp(); } - private void validate() + public void validate() { if (this.first.compareTo(this.last) > 0) { @@ -1314,7 +1453,7 @@ public KeyCacheKey getCacheKey(DecoratedKey key) return new KeyCacheKey(metadata(), descriptor, key.getKey()); } - public void cacheKey(DecoratedKey key, RowIndexEntry info) + public void cacheKey(DecoratedKey key, BigTableRowIndexEntry info) { CachingParams caching = metadata().params.caching; @@ -1326,20 +1465,20 @@ public void cacheKey(DecoratedKey key, RowIndexEntry info) keyCache.put(cacheKey, info); } - public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats) + public BigTableRowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats) { if (isKeyCacheEnabled()) return getCachedPosition(new KeyCacheKey(metadata(), descriptor, key.getKey()), updateStats); return null; } - protected RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats) + protected BigTableRowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats) { if (isKeyCacheEnabled()) { if (updateStats) { - RowIndexEntry cachedEntry = keyCache.get(unifiedKey); + BigTableRowIndexEntry cachedEntry = keyCache.get(unifiedKey); keyCacheRequest.incrementAndGet(); if (cachedEntry != null) { @@ -1369,26 +1508,14 @@ public boolean isKeyCacheEnabled() */ public final RowIndexEntry getPosition(PartitionPosition key, Operator op) { - return getPosition(key, op, SSTableReadsListener.NOOP_LISTENER); + return getPosition(key, op, true, false, SSTableReadsListener.NOOP_LISTENER); } - /** - * Retrieves the position while updating the key cache and the stats. - * @param key The key to apply as the rhs to the given Operator. A 'fake' key is allowed to - * allow key selection by token bounds but only if op != * EQ - * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins. - * @param listener the {@code SSTableReaderListener} that must handle the notifications. - */ - public final RowIndexEntry getPosition(PartitionPosition key, Operator op, SSTableReadsListener listener) - { - return getPosition(key, op, true, false, listener); - } - - public final RowIndexEntry getPosition(PartitionPosition key, - Operator op, - boolean updateCacheAndStats) + public final boolean checkEntryExists(PartitionPosition key, + Operator op, + boolean updateCacheAndStats) { - return getPosition(key, op, updateCacheAndStats, false, SSTableReadsListener.NOOP_LISTENER); + return getPosition(key, op, updateCacheAndStats, false, SSTableReadsListener.NOOP_LISTENER) != null; } /** @@ -1411,9 +1538,7 @@ public abstract UnfilteredRowIterator iterator(DecoratedKey key, boolean reversed, SSTableReadsListener listener); - public abstract UnfilteredRowIterator iterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, Slices slices, ColumnFilter selectedColumns, boolean reversed); - - public abstract UnfilteredRowIterator simpleIterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, boolean tombstoneOnly); + public abstract UnfilteredRowIterator simpleIterator(FileDataInput dfile, DecoratedKey key, boolean tombstoneOnly); /** * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists. @@ -1428,24 +1553,22 @@ public DecoratedKey firstKeyBeyond(PartitionPosition token) if (ifile == null) return null; - String path = null; - try (FileDataInput in = ifile.createReader(sampledPosition)) + try (PartitionIndexIterator iterator = allKeysIterator()) { - path = in.getPath(); - while (!in.isEOF()) + iterator.indexPosition(sampledPosition); + KeyIterator keyIterator = new KeyIterator(iterator, getPartitioner(), uncompressedLength()); + + while (keyIterator.hasNext()) { - ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in); - DecoratedKey indexDecoratedKey = decorateKey(indexKey); + DecoratedKey indexDecoratedKey = keyIterator.next(); if (indexDecoratedKey.compareTo(token) > 0) return indexDecoratedKey; - - RowIndexEntry.Serializer.skip(in, descriptor.version); } } catch (IOException e) { markSuspect(); - throw new CorruptSSTableException(e, path); + throw new CorruptSSTableException(e, ifile.path()); } return null; @@ -1618,24 +1741,106 @@ public boolean isRepaired() return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE; } - public DecoratedKey keyAt(long indexPosition) throws IOException + public DecoratedKey keyAt(RandomAccessReader reader, long position) throws IOException + { + reader.seek(position); + return keyAt(reader); + } + + public abstract DecoratedKey keyAt(FileDataInput reader) throws IOException; + + /** + * Retrieves the partition-level deletion time at the given position of the data file, as specified by + * {@link SSTableFlushObserver#partitionLevelDeletion(DeletionTime, long)}. + * + * @param position the start position of the partion-level deletion time in the data file + * @return the partion-level deletion time at the specified position + */ + public DeletionTime partitionLevelDeletionAt(long position) throws IOException + { + try (FileDataInput in = dfile.createReader(position)) + { + if (in.isEOF()) + return null; + + return DeletionTime.serializer.deserialize(in); + } + } + + /** + * Retrieves the static row at the given position of the data file, as specified by + * {@link SSTableFlushObserver#staticRow(Row, long)}. + * + * @param position the start position of the static row in the data file + * @param columnFilter the columns to fetch, {@code null} to select all the columns + * @return the static row at the specified position + */ + public Row staticRowAt(long position, ColumnFilter columnFilter) throws IOException { - DecoratedKey key; - try (FileDataInput in = ifile.createReader(indexPosition)) + if (!header.hasStatic()) + return Rows.EMPTY_STATIC_ROW; + + try (FileDataInput in = dfile.createReader(position)) { if (in.isEOF()) return null; - key = decorateKey(ByteBufferUtil.readWithShortLength(in)); + int version = descriptor.version.correspondingMessagingVersion(); + DeserializationHelper helper = new DeserializationHelper(metadata.get(), + version, + DeserializationHelper.Flag.LOCAL, + columnFilter); - // hint read path about key location if caching is enabled - // this saves index summary lookup and index file iteration which whould be pretty costly - // especially in presence of promoted column indexes - if (isKeyCacheEnabled()) - cacheKey(key, rowIndexEntrySerializer.deserialize(in)); + return UnfilteredSerializer.serializer.deserializeStaticRow(in, header, helper); } + } - return key; + /** + * Retrieves the clustering prefix of the unfiltered at the given position of the data file, as specified by + * {@link SSTableFlushObserver#nextUnfilteredCluster(Unfiltered, long)}. + * + * @param position the start position of the unfiltered in the data file + * @return the clustering prefix of the unfiltered at the specified position + */ + public ClusteringPrefix clusteringAt(long position) throws IOException + { + try (FileDataInput in = dfile.createReader(position)) + { + if (in.isEOF()) + return null; + + int version = descriptor.version.correspondingMessagingVersion(); + int flags = in.readUnsignedByte(); + boolean isRow = UnfilteredSerializer.kind(flags) == Unfiltered.Kind.ROW; + + return isRow + ? Clustering.serializer.deserialize(in, version, header.clusteringTypes()) + : ClusteringBoundOrBoundary.serializer.deserialize(in, version, header.clusteringTypes()); + } + } + + /** + * Retrieves the unfiltered at the given position of the data file, as specified by + * {@link SSTableFlushObserver#nextUnfilteredCluster(Unfiltered, long)}. + * + * @param position the start position of the unfiltered in the data file + * @param columnFilter the columns to fetch, {@code null} to select all the columns + * @return the unfiltered at the specified position + */ + public Unfiltered unfilteredAt(long position, ColumnFilter columnFilter) throws IOException + { + try (FileDataInput in = dfile.createReader(position)) + { + if (in.isEOF()) + return null; + + int version = descriptor.version.correspondingMessagingVersion(); + DeserializationHelper helper = new DeserializationHelper(metadata.get(), + version, + DeserializationHelper.Flag.LOCAL, + columnFilter); + return UnfilteredSerializer.serializer.deserialize(in, header, helper, BTreeRow.sortedBuilder()); + } } public boolean isPendingRepair() @@ -1695,37 +1900,7 @@ final static class GreaterThan extends Operator } } - public long getBloomFilterFalsePositiveCount() - { - return bloomFilterTracker.getFalsePositiveCount(); - } - - public long getRecentBloomFilterFalsePositiveCount() - { - return bloomFilterTracker.getRecentFalsePositiveCount(); - } - - public long getBloomFilterTruePositiveCount() - { - return bloomFilterTracker.getTruePositiveCount(); - } - - public long getRecentBloomFilterTruePositiveCount() - { - return bloomFilterTracker.getRecentTruePositiveCount(); - } - - public long getBloomFilterTrueNegativeCount() - { - return bloomFilterTracker.getTrueNegativeCount(); - } - - public long getRecentBloomFilterTrueNegativeCount() - { - return bloomFilterTracker.getRecentTrueNegativeCount(); - } - - public InstrumentingCache getKeyCache() + public InstrumentingCache getKeyCache() { return keyCache; } @@ -1882,6 +2057,8 @@ public RandomAccessReader openIndexReader() return null; } + public abstract RandomAccessReader openKeyComponentReader(); + public ChannelProxy getDataChannel() { return dfile.channel; @@ -1954,9 +2131,8 @@ public Ref ref() return selfRef.ref(); } - void setup(boolean trackHotness) + public void setup(boolean trackHotness) { - tidy.setup(this, trackHotness); this.readMeter = tidy.global.readMeter; } @@ -1986,15 +2162,12 @@ public void addTo(Ref.IdentityCollection identities) * When the InstanceTidier cleansup, it releases its reference to its GlobalTidy; when all InstanceTidiers * for that type have run, the GlobalTidy cleans up. */ - private static final class InstanceTidier implements Tidy + protected static final class InstanceTidier implements RefCounted.Tidy { private final Descriptor descriptor; private final TableId tableId; - private IFilter bf; - private IndexSummary summary; - private FileHandle dfile; - private FileHandle ifile; + private List closables; private Runnable runOnClose; private boolean isReplaced = false; @@ -2005,18 +2178,15 @@ private static final class InstanceTidier implements Tidy private volatile boolean setup; - void setup(SSTableReader reader, boolean trackHotness) + public void setup(SSTableReader reader, boolean trackHotness, List closables) { this.setup = true; - this.bf = reader.bf; - this.summary = reader.indexSummary; - this.dfile = reader.dfile; - this.ifile = reader.ifile; // get a new reference to the shared descriptor-type tidy this.globalRef = GlobalTidy.get(reader); this.global = globalRef.get(); if (trackHotness) global.ensureReadMeter(); + this.closables = closables; } InstanceTidier(Descriptor descriptor, TableId tableId) @@ -2044,34 +2214,49 @@ public void tidy() else barrier = null; - ScheduledExecutors.nonPeriodicTasks.execute(new Runnable() - { - public void run() + ScheduledExecutors.nonPeriodicTasks.execute(() -> { + if (logger.isTraceEnabled()) + logger.trace("Async instance tidier for {}, before barrier", descriptor); + + if (barrier != null) + barrier.await(); + + if (logger.isTraceEnabled()) + logger.trace("Async instance tidier for {}, after barrier", descriptor); + + Throwable exceptions = null; + if (runOnClose != null) try { - if (logger.isTraceEnabled()) - logger.trace("Async instance tidier for {}, before barrier", descriptor); - - if (barrier != null) - barrier.await(); - - if (logger.isTraceEnabled()) - logger.trace("Async instance tidier for {}, after barrier", descriptor); - - if (bf != null) - bf.close(); - if (summary != null) - summary.close(); - if (runOnClose != null) - runOnClose.run(); - if (dfile != null) - dfile.close(); - if (ifile != null) - ifile.close(); - globalRef.release(); + runOnClose.run(); + } + catch (RuntimeException | Error ex) + { + logger.error("Failed to run on-close listeners for sstable " + descriptor.baseFilename(), ex); + exceptions = ex; + } + + Throwable closeExceptions = Throwables.close(null, closables); + if (closeExceptions != null) + { + logger.error("Failed to close some sstable components of " + descriptor.baseFilename(), closeExceptions); + exceptions = Throwables.merge(exceptions, closeExceptions); + } - if (logger.isTraceEnabled()) - logger.trace("Async instance tidier for {}, completed", descriptor); + try + { + globalRef.release(); + } + catch (RuntimeException | Error ex) + { + logger.error("Failed to release the global ref of " + descriptor.baseFilename(), ex); + exceptions = Throwables.merge(exceptions, ex); } + + if (exceptions != null) + JVMStabilityInspector.inspectThrowable(exceptions); + + if (logger.isTraceEnabled()) + logger.trace("Async instance tidier for {}, completed", descriptor); }); } @@ -2080,12 +2265,6 @@ public String name() return descriptor.toString(); } - void releaseSummary() - { - summary.close(); - assert summary.isCleanedUp(); - summary = null; - } } /** @@ -2096,7 +2275,7 @@ void releaseSummary() * and stash a reference to it to be released when they are. Once all such references are * released, this shared tidy will be performed. */ - static final class GlobalTidy implements Tidy + static final class GlobalTidy implements RefCounted.Tidy { static final WeakReference> NULL = new WeakReference<>(null); // keyed by descriptor, mapping to the shared GlobalTidy for that descriptor @@ -2200,11 +2379,97 @@ public static void resetTidying() GlobalTidy.lookup.clear(); } - public static abstract class Factory + /** + * Main entry point for opening (creating a new instance of) a Reader. The desired usage is obtaining the + * factory instance from SSTable descriptor and invoking one of the open methods. This usage makes + * static @{link SSTableReader} open* methods obsolete (all of them are private now). + * {@link SSTableReader} subclasses are exepected to provide an implementation of this interface. + */ + public interface Factory { - public abstract SSTableReader open(SSTableReaderBuilder builder); + PartitionIndexIterator indexIterator(Descriptor descriptor, TableMetadata metadata); + + // TODO in the implementation of those methods we will refer the current static methods which are implemented in AbstractdBigTableReader + // TODO make those static openXXX methods private + + SSTableReader openForBatch(Descriptor desc, Set components, TableMetadataRef metadata); + + SSTableReader open(Descriptor desc); + + SSTableReader open(Descriptor desc, TableMetadataRef metadata); + + SSTableReader open(Descriptor desc, Set components, TableMetadataRef metadata); + + SSTableReader open(Descriptor desc, Set components, TableMetadataRef metadata, boolean validate, boolean isOffline); + + SSTableReader openNoValidation(Descriptor desc, TableMetadataRef tableMetadataRef); + + SSTableReader openNoValidation(Descriptor desc, Set components, ColumnFamilyStore cfs); + + SSTableReader moveAndOpenSSTable(ColumnFamilyStore cfs, Descriptor oldDescriptor, Descriptor newDescriptor, Set components, boolean copyData); + } + /** + * Opens BigTable format readers. Proxies open calls to (private) static methods of @{link SSTableReader}. + * This class servers as a proxy to legacy private static methods that should be refactored out of + * SSTableReader (as they are specific to BigTable format) but are left in the Reader for painless + * upstream merges. + * Implementations of this abstract class is provided by BigTable format reader. + */ + public abstract static class AbstractBigTableReaderFactory implements SSTableReader.Factory + { + @Override + public SSTableReader openForBatch(Descriptor desc, Set components, TableMetadataRef metadata) + { + return SSTableReader.openForBatch(desc, components, metadata); + } + + @Override + public SSTableReader open(Descriptor desc) + { + return SSTableReader.open(desc); + } + + @Override + public SSTableReader open(Descriptor desc, TableMetadataRef metadata) + { + return SSTableReader.open(desc, metadata); + } + + @Override + public SSTableReader open(Descriptor desc, Set components, TableMetadataRef metadata) + { + return SSTableReader.open(desc, components, metadata); + } + + @Override + public SSTableReader open(Descriptor desc, Set components, TableMetadataRef metadata, boolean validate, boolean isOffline) + { + return SSTableReader.open(desc, components, metadata, validate, isOffline); + } + + @Override + public SSTableReader openNoValidation(Descriptor desc, TableMetadataRef tableMetadataRef) + { + return SSTableReader.openNoValidation(desc, tableMetadataRef); + } + + @Override + public SSTableReader openNoValidation(Descriptor desc, Set components, ColumnFamilyStore cfs) + { + return SSTableReader.openNoValidation(desc, components, cfs); + } + + @Override + public SSTableReader moveAndOpenSSTable(ColumnFamilyStore cfs, Descriptor oldDescriptor, Descriptor newDescriptor, Set components, boolean copyData) + { + return SSTableReader.moveAndOpenSSTable(cfs, oldDescriptor, newDescriptor, components, copyData); + } + + } + + public static class PartitionPositionBounds { public final long lowerPosition; @@ -2309,7 +2574,7 @@ public static SSTableReader moveAndOpenSSTable(ColumnFamilyStore cfs, Descriptor SSTableReader reader; try { - reader = SSTableReader.open(newDescriptor, components, cfs.metadata); + reader = newDescriptor.formatType.info.getReaderFactory().open(newDescriptor, components, cfs.metadata); } catch (Throwable t) { @@ -2325,4 +2590,91 @@ public static void shutdownBlocking(long timeout, TimeUnit unit) throws Interrup ExecutorUtils.shutdownNowAndWait(timeout, unit, syncExecutor); resetTidying(); } + + public static void checkRequiredComponents(Descriptor descriptor, Set components, boolean validate) + { + if (validate) + { + Set requiredComponents = descriptor.formatType.info.requiredComponents(); + // Minimum components without which we can't do anything + assert components.containsAll(requiredComponents) : String.format("Required components %s missing for sstable %s", Sets.difference(requiredComponents, components), descriptor); + } + else + { + // Scrub-only case, we just need data file. + assert components.contains(Component.DATA); + } + } + + public static @Nonnull boolean shouldLoadBloomFilter(Descriptor desc, Set components, double currerntFPChance, double desiredFPChance) + { + if (!BloomFilter.shouldUseBloomFilter(desiredFPChance)) + { + if (logger.isTraceEnabled()) + logger.trace("Bloom filter for {} will not be loaded because fpChance={} is neglectable", desc, desiredFPChance); + + return false; + } + else if (!components.containsAll(desc.getFormat().primaryIndexComponents())) + { + if (logger.isTraceEnabled()) + logger.trace("Bloom filter for {} will not be loaded because there are missing primary index components: {}", desc, Sets.difference(desc.getFormat().primaryIndexComponents(), components)); + + return false; + } + else if (!components.contains(Component.FILTER) || Double.isNaN(currerntFPChance)) + { + if (logger.isTraceEnabled()) + logger.trace("Bloom filter for {} will not be loaded because filter component is missing or sstable lacks validation metadata", desc); + + return false; + } + else if (!BloomFilter.isFPChanceDiffNeglectable(desiredFPChance, currerntFPChance) && BloomFilter.recreateOnFPChanceChange) + { + if (logger.isTraceEnabled()) + logger.trace("Bloom filter for {} will not be loaded because fpChance has changed from {} to {} and the filter should be recreated", desc, currerntFPChance, desiredFPChance); + + return false; + } + + return true; + } + + public static boolean mayRecreateBloomFilter(Descriptor desc, Set components, double currentFPChance, boolean isOffline, double desiredFPChance) + { + if (!BloomFilter.shouldUseBloomFilter(desiredFPChance)) + { + if (logger.isTraceEnabled()) + logger.trace("Bloom filter for {} must not be recreated because fpChance={} is neglectable", desc, desiredFPChance); + + return false; + } + else if (!components.containsAll(desc.getFormat().primaryIndexComponents())) + { + if (logger.isTraceEnabled()) + logger.trace("Bloom filter for {} must not be recreated because there are missing primary index components: {}", desc, Sets.difference(desc.getFormat().primaryIndexComponents(), components)); + + return false; + } + else if (!components.contains(Component.FILTER) || Double.isNaN(currentFPChance)) + { + if (logger.isTraceEnabled() && isOffline) + logger.trace("Bloom filter for {} must not be recreated because sstable has been opened in offline mode", desc); + + return !isOffline; + } + else if (!BloomFilter.isFPChanceDiffNeglectable(desiredFPChance, currentFPChance) && BloomFilter.recreateOnFPChanceChange) + { + if (logger.isTraceEnabled() && isOffline) + logger.trace("Bloom filter for {} must not be recreated because sstable has been opened in offline mode", desc); + + return !isOffline; + } + else + { + // bf is enabled and fp chance matches the currently configured value. + return true; + } + } + } diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java index 8fe1deff9e4e..6f18705bbaed 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReaderBuilder.java @@ -18,34 +18,36 @@ package org.apache.cassandra.io.sstable.format; +import java.io.BufferedInputStream; +import java.io.DataInputStream; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.cache.ChunkCache; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.RowIndexEntry; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.io.sstable.*; +import org.apache.cassandra.io.sstable.format.big.BigTableReader; +import org.apache.cassandra.io.sstable.metadata.MetadataType; import org.apache.cassandra.io.sstable.metadata.StatsMetadata; import org.apache.cassandra.io.sstable.metadata.ValidationMetadata; import org.apache.cassandra.io.util.DiskOptimizationStrategy; import org.apache.cassandra.io.util.FileHandle; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.io.util.RandomAccessReader; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.utils.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.BufferedInputStream; -import java.io.DataInputStream; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Set; -import java.util.concurrent.TimeUnit; public abstract class SSTableReaderBuilder { @@ -89,28 +91,20 @@ public SSTableReaderBuilder(Descriptor descriptor, public abstract SSTableReader build(); - public SSTableReaderBuilder dfile(FileHandle dfile) - { - this.dfile = dfile; - return this; - } - - public SSTableReaderBuilder ifile(FileHandle ifile) + @SuppressWarnings("resource") + public static FileHandle.Builder defaultIndexHandleBuilder(Descriptor descriptor, Component component) { - this.ifile = ifile; - return this; + return new FileHandle.Builder(descriptor.filenameFor(component)) + .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap) + .withChunkCache(ChunkCache.instance); } - public SSTableReaderBuilder bf(IFilter bf) + @SuppressWarnings("resource") + public static FileHandle.Builder defaultDataHandleBuilder(Descriptor descriptor) { - this.bf = bf; - return this; - } - - public SSTableReaderBuilder summary(IndexSummary summary) - { - this.summary = summary; - return this; + return new FileHandle.Builder(descriptor.filenameFor(Component.DATA)) + .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap) + .withChunkCache(ChunkCache.instance); } /** @@ -171,47 +165,43 @@ void buildSummaryAndBloomFilter(boolean recreateBloomFilter, if (!components.contains(Component.PRIMARY_INDEX)) return; + if (!recreateBloomFilter && summaryLoaded) + return; + if (logger.isDebugEnabled()) logger.debug("Attempting to build summary for {}", descriptor); - - // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary. - try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)))) - { - long indexSize = primaryIndex.length(); + try (PartitionIndexIterator indexIterator = readerFactory.indexIterator(descriptor, metadata)) { long histogramCount = statsMetadata.estimatedPartitionSize.count(); long estimatedKeys = histogramCount > 0 && !statsMetadata.estimatedPartitionSize.isOverflowed() ? histogramCount - : SSTable.estimateRowsFromIndex(primaryIndex, descriptor); // statistics is supposed to be optional - + : SSTable.estimateRowsFromIndex(indexIterator); // statistics is supposed to be optional if (recreateBloomFilter) + { + logger.debug("Recreating bloom filter for {} with fpChance={}", descriptor, metadata.params.bloomFilterFpChance); bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance); + } + // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary. try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL)) { - long indexPosition; - - while ((indexPosition = primaryIndex.getFilePointer()) != indexSize) + while (!indexIterator.isExhausted()) { - ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex); - RowIndexEntry.Serializer.skip(primaryIndex, descriptor.version); - DecoratedKey decoratedKey = metadata.partitioner.decorateKey(key); + DecoratedKey decoratedKey = metadata.partitioner.decorateKey(indexIterator.key()); if (!summaryLoaded) { if (first == null) first = decoratedKey; last = decoratedKey; + + summaryBuilder.maybeAddEntry(decoratedKey, indexIterator.keyPosition()); } if (recreateBloomFilter) bf.add(decoratedKey); - // if summary was already read from disk we don't want to re-populate it using primary index - if (!summaryLoaded) - { - summaryBuilder.maybeAddEntry(decoratedKey, indexPosition); - } + indexIterator.advance(); } if (!summaryLoaded) @@ -226,17 +216,30 @@ void buildSummaryAndBloomFilter(boolean recreateBloomFilter, } } - /** - * Load bloom filter from Filter.db file. - * - * @throws IOException - */ - IFilter loadBloomFilter() throws IOException + public static IFilter loadBloomFilter(Path path, boolean oldFormat) { - try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(Paths.get(descriptor.filenameFor(Component.FILTER)))))) + if (Files.exists(path)) + { + logger.debug("Loading bloom filter from {}", path); + IFilter filter = null; + try (DataInputStream stream = new DataInputStream(new BufferedInputStream(Files.newInputStream(path)))) + { + filter = BloomFilter.serializer.deserialize(stream, oldFormat); + return filter; + } + catch (Throwable t) + { + JVMStabilityInspector.inspectThrowable(t); + logger.error("Failed to deserialize Bloom filter: {}", t.getMessage()); + if (filter != null) + filter.close(); + } + } + else { - return BloomFilterSerializer.deserialize(stream, descriptor.version.hasOldBfFormat()); + logger.error("Bloom filter {} not found", path); } + return null; } public static class ForWriter extends SSTableReaderBuilder @@ -252,10 +255,34 @@ public ForWriter(Descriptor descriptor, super(descriptor, metadataRef, maxDataAge, components, statsMetadata, openReason, header); } + public SSTableReaderBuilder.ForWriter dfile(FileHandle dfile) + { + this.dfile = dfile; + return this; + } + + public SSTableReaderBuilder.ForWriter ifile(FileHandle ifile) + { + this.ifile = ifile; + return this; + } + + public SSTableReaderBuilder.ForWriter bf(IFilter bf) + { + this.bf = bf; + return this; + } + + public SSTableReaderBuilder.ForWriter summary(IndexSummary summary) + { + this.summary = summary; + return this; + } + @Override public SSTableReader build() { - SSTableReader reader = readerFactory.open(this); + SSTableReader reader = new BigTableReader(this); reader.setup(true); return reader; @@ -276,6 +303,7 @@ public ForBatch(Descriptor descriptor, @Override public SSTableReader build() { + assert dfile == null && ifile == null && summary == null && bf == null; String dataFilePath = descriptor.filenameFor(Component.DATA); long fileLength = new File(dataFilePath).length(); logger.info("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength)); @@ -283,12 +311,8 @@ public SSTableReader build() initSummary(dataFilePath, components, statsMetadata); boolean compression = components.contains(Component.COMPRESSION_INFO); - 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)) + try (FileHandle.Builder ibuilder = defaultIndexHandleBuilder(descriptor, Component.PRIMARY_INDEX); + FileHandle.Builder dbuilder = defaultDataHandleBuilder(descriptor).compressed(compression)) { long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length(); DiskOptimizationStrategy optimizationStrategy = DatabaseDescriptor.getDiskOptimizationStrategy(); @@ -298,7 +322,7 @@ public SSTableReader build() dfile = dbuilder.bufferSize(dataBufferSize).complete(); bf = FilterFactory.AlwaysPresent; - SSTableReader sstable = readerFactory.open(this); + SSTableReader sstable = new BigTableReader(this); sstable.first = first; sstable.last = last; @@ -327,7 +351,7 @@ void initSummary(String dataFilePath, Set components, StatsMetadata s public static class ForRead extends SSTableReaderBuilder { - private final ValidationMetadata validationMetadata; + private volatile ValidationMetadata validationMetadata; private final boolean isOffline; public ForRead(Descriptor descriptor, @@ -346,6 +370,7 @@ public ForRead(Descriptor descriptor, @Override public SSTableReader build() { + assert dfile == null && ifile == null && summary == null && bf == null; String dataFilePath = descriptor.filenameFor(Component.DATA); long fileLength = new File(dataFilePath).length(); logger.info("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength)); @@ -362,7 +387,7 @@ public SSTableReader build() throw new CorruptSSTableException(t, dataFilePath); } - SSTableReader sstable = readerFactory.open(this); + SSTableReader sstable = new BigTableReader(this); sstable.first = first; sstable.last = last; @@ -381,30 +406,21 @@ private void load(ValidationMetadata validation, DiskOptimizationStrategy optimizationStrategy, StatsMetadata statsMetadata) throws IOException { - if (metadata.params.bloomFilterFpChance == 1.0) + double currentFPChance = validation != null ? validation.bloomFilterFPChance : Double.NaN; + double desiredFPChance = metadata.params.bloomFilterFpChance; + + if (SSTableReader.shouldLoadBloomFilter(descriptor, components, currentFPChance, desiredFPChance)) + bf = loadBloomFilter(Paths.get(descriptor.filenameFor(Component.FILTER)), descriptor.version.hasOldBfFormat()); + + boolean recreateBloomFilter = bf == null && SSTableReader.mayRecreateBloomFilter(descriptor, components, currentFPChance, isOffline, desiredFPChance); + load(recreateBloomFilter, !isOffline, optimizationStrategy, statsMetadata, components); + + // if the filter was neither loaded nor created, or we encountered some problems, we fallback to pass-through filter + if (bf == null) { - // bf is disabled. - load(false, !isOffline, optimizationStrategy, statsMetadata, components); bf = FilterFactory.AlwaysPresent; - } - else if (!components.contains(Component.PRIMARY_INDEX)) // What happens if filter component and primary index is missing? - { - // avoid any reading of the missing primary index component. - // this should only happen during StandaloneScrubber - load(false, !isOffline, optimizationStrategy, statsMetadata, components); - } - else if (!components.contains(Component.FILTER) || validation == null) - { - // bf is enabled, but filter component is missing. - load(!isOffline, !isOffline, optimizationStrategy, statsMetadata, components); - if (isOffline) - bf = FilterFactory.AlwaysPresent; - } - else - { - // bf is enabled and fp chance matches the currently configured value. - load(false, !isOffline, optimizationStrategy, statsMetadata, components); - bf = loadBloomFilter(); + logger.warn("Could not recreate or deserialize existing bloom filter, continuing with a pass-through " + + "bloom filter but this will significantly impact reads performance"); } } @@ -420,12 +436,9 @@ void load(boolean recreateBloomFilter, StatsMetadata statsMetadata, Set components) throws IOException { - 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(components.contains(Component.COMPRESSION_INFO)) - .mmapped(DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap) - .withChunkCache(ChunkCache.instance)) + boolean compression = components.contains(Component.COMPRESSION_INFO); + try (FileHandle.Builder ibuilder = defaultIndexHandleBuilder(descriptor, Component.PRIMARY_INDEX); + FileHandle.Builder dbuilder = defaultDataHandleBuilder(descriptor).compressed(compression)) { loadSummary(); boolean buildSummary = summary == null || recreateBloomFilter; @@ -448,7 +461,12 @@ void load(boolean recreateBloomFilter, if (saveSummaryIfCreated) SSTableReader.saveSummary(descriptor, first, last, summary); if (recreateBloomFilter) + { SSTableReader.saveBloomFilter(descriptor, bf); + ValidationMetadata updatedValidationMetadata = new ValidationMetadata(validationMetadata.partitioner, metadata.params.bloomFilterFpChance); + descriptor.getMetadataSerializer().updateSSTableMetadata(descriptor, ImmutableMap.of(MetadataType.VALIDATION, updatedValidationMetadata)); + validationMetadata = updatedValidationMetadata; + } } } catch (Throwable t) @@ -463,6 +481,12 @@ void load(boolean recreateBloomFilter, dfile.close(); } + if (bf != null) + { + bf.close(); + bf = null; + } + if (summary != null) { summary.close(); diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReadsListener.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReadsListener.java index 6d384bfb7233..db5d0398d3c1 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReadsListener.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReadsListener.java @@ -17,8 +17,6 @@ */ package org.apache.cassandra.io.sstable.format; -import org.apache.cassandra.db.RowIndexEntry; - /** * Listener for receiving notifications associated with reading SSTables. */ @@ -66,7 +64,7 @@ default void onSSTableSkipped(SSTableReader sstable, SkippingReason reason) * @param indexEntry the index entry * @param reason the reason for which the SSTable has been selected */ - default void onSSTableSelected(SSTableReader sstable, RowIndexEntry indexEntry, SelectionReason reason) + default void onSSTableSelected(SSTableReader sstable, RowIndexEntry indexEntry, SelectionReason reason) { } 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 43c50c56141a..21bd73215a4c 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java @@ -18,18 +18,27 @@ package org.apache.cassandra.io.sstable.format; +import java.nio.ByteBuffer; import java.util.*; +import java.util.function.Consumer; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.RowIndexEntry; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionPurger; import org.apache.cassandra.db.SerializationHeader; -import org.apache.cassandra.db.compaction.OperationType; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; +import org.apache.cassandra.db.rows.ComplexColumnData; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.guardrails.Guardrails; import org.apache.cassandra.index.Index; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.sstable.Component; @@ -40,9 +49,12 @@ import org.apache.cassandra.io.sstable.metadata.MetadataType; import org.apache.cassandra.io.sstable.metadata.StatsMetadata; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.Transactional; /** @@ -53,13 +65,14 @@ */ public abstract class SSTableWriter extends SSTable implements Transactional { + private static final Logger logger = LoggerFactory.getLogger(SSTableWriter.class); + protected long repairedAt; protected UUID pendingRepair; protected boolean isTransient; protected long maxDataAge = -1; protected final long keyCount; protected final MetadataCollector metadataCollector; - protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer; protected final SerializationHeader header; protected final TransactionalProxy txnProxy = txnProxy(); protected final Collection observers; @@ -75,6 +88,7 @@ protected abstract class TransactionalProxy extends AbstractTransactional } protected SSTableWriter(Descriptor descriptor, + Set components, long keyCount, long repairedAt, UUID pendingRepair, @@ -84,17 +98,30 @@ protected SSTableWriter(Descriptor descriptor, SerializationHeader header, Collection observers) { - super(descriptor, components(metadata.getLocal()), metadata, DatabaseDescriptor.getDiskOptimizationStrategy()); + super(descriptor, components, metadata, DatabaseDescriptor.getDiskOptimizationStrategy()); this.keyCount = keyCount; this.repairedAt = repairedAt; this.pendingRepair = pendingRepair; this.isTransient = isTransient; this.metadataCollector = metadataCollector; this.header = header; - this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata.get(), descriptor.version, header); this.observers = observers == null ? Collections.emptySet() : observers; } + private static Set indexComponents(Collection indexGroups) + { + if (indexGroups == null) + return Collections.emptySet(); + + Set components = new HashSet<>(); + for (Index.Group group : indexGroups) + { + components.addAll(group.getComponents()); + } + + return components; + } + public static SSTableWriter create(Descriptor descriptor, Long keyCount, Long repairedAt, @@ -103,11 +130,12 @@ public static SSTableWriter create(Descriptor descriptor, TableMetadataRef metadata, MetadataCollector metadataCollector, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { Factory writerFactory = descriptor.getFormat().getWriterFactory(); - return writerFactory.open(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers(descriptor, indexes, lifecycleNewTracker.opType()), lifecycleNewTracker); + return writerFactory.open(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers(descriptor, indexGroups, lifecycleNewTracker, metadata.get()), lifecycleNewTracker, + indexComponents(indexGroups)); } public static SSTableWriter create(Descriptor descriptor, @@ -117,11 +145,11 @@ public static SSTableWriter create(Descriptor descriptor, boolean isTransient, int sstableLevel, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { TableMetadataRef metadata = Schema.instance.getTableMetadataRef(descriptor); - return create(metadata, descriptor, keyCount, repairedAt, pendingRepair, isTransient, sstableLevel, header, indexes, lifecycleNewTracker); + return create(metadata, descriptor, keyCount, repairedAt, pendingRepair, isTransient, sstableLevel, header, indexGroups, lifecycleNewTracker); } public static SSTableWriter create(TableMetadataRef metadata, @@ -132,11 +160,11 @@ public static SSTableWriter create(TableMetadataRef metadata, boolean isTransient, int sstableLevel, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(sstableLevel); - return create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, collector, header, indexes, lifecycleNewTracker); + return create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, collector, header, indexGroups, lifecycleNewTracker); } @VisibleForTesting @@ -146,13 +174,16 @@ public static SSTableWriter create(Descriptor descriptor, UUID pendingRepair, boolean isTransient, SerializationHeader header, - Collection indexes, + Collection indexGroups, LifecycleNewTracker lifecycleNewTracker) { - return create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, 0, header, indexes, lifecycleNewTracker); + return create(descriptor, keyCount, repairedAt, pendingRepair, isTransient, 0, header, indexGroups, lifecycleNewTracker); } - private static Set components(TableMetadata metadata) + /** + * BigTable SSTable components. Should be moved to BigTableWriter but is left here for painless upstream merges. + */ + public static Set bigTableComponents(TableMetadata metadata, Collection indexComponents) { Set components = new HashSet(Arrays.asList(Component.DATA, Component.PRIMARY_INDEX, @@ -174,20 +205,24 @@ private static Set components(TableMetadata metadata) // but the components are unmodifiable after construction components.add(Component.CRC); } + + components.addAll(indexComponents); + return components; } private static Collection observers(Descriptor descriptor, - Collection indexes, - OperationType operationType) + Collection indexGroups, + LifecycleNewTracker tracker, + TableMetadata metadata) { - if (indexes == null) + if (indexGroups == null) return Collections.emptyList(); - List observers = new ArrayList<>(indexes.size()); - for (Index index : indexes) + List observers = new ArrayList<>(indexGroups.size()); + for (Index.Group group : indexGroups) { - SSTableFlushObserver observer = index.getFlushObserver(descriptor, operationType); + SSTableFlushObserver observer = group.getFlushObserver(descriptor, tracker, metadata); if (observer != null) { observer.begin(); @@ -244,7 +279,7 @@ public SSTableWriter setOpenResult(boolean openResult) /** * Open the resultant SSTableReader before it has been fully written */ - public abstract SSTableReader openEarly(); + public abstract boolean openEarly(Consumer callWhenReady); /** * Open the resultant SSTableReader once it has been fully written, but before the @@ -297,7 +332,14 @@ public final Throwable commit(Throwable accumulate) public final Throwable abort(Throwable accumulate) { - return txnProxy.abort(accumulate); + try + { + return txnProxy.abort(accumulate); + } + finally + { + observers.forEach(observer -> observer.abort(accumulate)); + } } public final void close() @@ -307,7 +349,14 @@ public final void close() public final void abort() { - txnProxy.abort(); + try + { + txnProxy.abort(); + } + finally + { + observers.forEach(observer -> observer.abort(null)); + } } protected Map finalizeMetadata() @@ -341,7 +390,8 @@ public static void rename(Descriptor tmpdesc, Descriptor newdesc, Set FileUtils.renameWithConfirm(tmpdesc.filenameFor(Component.DATA), newdesc.filenameFor(Component.DATA)); // rename it without confirmation because summary can be available for loadNewSSTables but not for closeAndOpenReader - FileUtils.renameWithOutConfirm(tmpdesc.filenameFor(Component.SUMMARY), newdesc.filenameFor(Component.SUMMARY)); + if (components.contains(Component.SUMMARY)) + FileUtils.renameWithOutConfirm(tmpdesc.filenameFor(Component.SUMMARY), newdesc.filenameFor(Component.SUMMARY)); } public static void copy(Descriptor tmpdesc, Descriptor newdesc, Set components) @@ -355,7 +405,8 @@ public static void copy(Descriptor tmpdesc, Descriptor newdesc, Set c FileUtils.copyWithConfirm(tmpdesc.filenameFor(Component.DATA), newdesc.filenameFor(Component.DATA)); // copy it without confirmation because summary can be available for loadNewSSTables but not for closeAndOpenReader - FileUtils.copyWithOutConfirm(tmpdesc.filenameFor(Component.SUMMARY), newdesc.filenameFor(Component.SUMMARY)); + if (components.contains(Component.SUMMARY)) + FileUtils.copyWithOutConfirm(tmpdesc.filenameFor(Component.SUMMARY), newdesc.filenameFor(Component.SUMMARY)); } public static void hardlink(Descriptor tmpdesc, Descriptor newdesc, Set components) @@ -369,11 +420,62 @@ public static void hardlink(Descriptor tmpdesc, Descriptor newdesc, Set observers, - LifecycleNewTracker lifecycleNewTracker); + LifecycleNewTracker lifecycleNewTracker, + Set indexComponents); + } + + protected void maybeLogLargePartitionWarning(DecoratedKey key, long rowSize) + { + if (SchemaConstants.isInternalKeyspace(metadata().keyspace)) + return; + + if (Guardrails.partitionSize.triggersOn(rowSize, null)) + { + String keyString = metadata().partitionKeyAsCQLLiteral(key.getKey()); + Guardrails.partitionSize.guard(rowSize, String.format("%s in %s", keyString, metadata), true, null); + } } } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableZeroCopyWriter.java similarity index 84% rename from src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java rename to src/java/org/apache/cassandra/io/sstable/format/SSTableZeroCopyWriter.java index f05ea94cb7ea..ee72094fcd7e 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableZeroCopyWriter.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.io.sstable.format.big; +package org.apache.cassandra.io.sstable.format; import java.io.EOFException; import java.io.File; @@ -39,7 +39,6 @@ import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SSTable; import org.apache.cassandra.io.sstable.SSTableMultiWriter; -import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.SequentialWriter; import org.apache.cassandra.io.util.SequentialWriterOption; @@ -50,11 +49,10 @@ import static java.lang.String.format; import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; -public class BigTableZeroCopyWriter extends SSTable implements SSTableMultiWriter +public class SSTableZeroCopyWriter extends SSTable implements SSTableMultiWriter { - private static final Logger logger = LoggerFactory.getLogger(BigTableZeroCopyWriter.class); + private static final Logger logger = LoggerFactory.getLogger(SSTableZeroCopyWriter.class); - private final TableMetadataRef metadata; private volatile SSTableReader finalReader; private final Map componentWriters; @@ -65,30 +63,19 @@ public class BigTableZeroCopyWriter extends SSTable implements SSTableMultiWrite .bufferType(BufferType.OFF_HEAP) .build(); - private static final ImmutableSet SUPPORTED_COMPONENTS = - ImmutableSet.of(Component.DATA, - Component.PRIMARY_INDEX, - Component.SUMMARY, - Component.STATS, - Component.COMPRESSION_INFO, - Component.FILTER, - Component.DIGEST, - Component.CRC); - - public BigTableZeroCopyWriter(Descriptor descriptor, - TableMetadataRef metadata, - LifecycleNewTracker lifecycleNewTracker, - final Collection components) + public SSTableZeroCopyWriter(Descriptor descriptor, + TableMetadataRef metadata, + LifecycleNewTracker lifecycleNewTracker, + final Collection components) { super(descriptor, ImmutableSet.copyOf(components), metadata, DatabaseDescriptor.getDiskOptimizationStrategy()); lifecycleNewTracker.trackNew(this); - this.metadata = metadata; this.componentWriters = new EnumMap<>(Component.Type.class); - if (!SUPPORTED_COMPONENTS.containsAll(components)) + if (!descriptor.getFormat().streamingComponents().containsAll(components)) throw new AssertionError(format("Unsupported streaming component detected %s", - Sets.difference(ImmutableSet.copyOf(components), SUPPORTED_COMPONENTS))); + Sets.difference(ImmutableSet.copyOf(components), descriptor.getFormat().streamingComponents()))); for (Component c : components) componentWriters.put(c.type, makeWriter(descriptor, c)); @@ -145,7 +132,7 @@ public Collection finish(boolean openResult) public Collection finished() { if (finalReader == null) - finalReader = SSTableReader.open(descriptor, components, metadata); + finalReader = descriptor.getFormat().getReaderFactory().open(descriptor, components, metadata); return ImmutableList.of(finalReader); } @@ -157,11 +144,24 @@ public SSTableMultiWriter setOpenResult(boolean openResult) } @Override - public long getFilePointer() + public long getBytesWritten() { + // TODO: these two may need fixing. return 0; } + @Override + public long getOnDiskBytesWritten() + { + return 0; + } + + @Override + public int getSegmentCount() + { + return 1; + } + @Override public TableId getTableId() { diff --git a/src/java/org/apache/cassandra/io/sstable/format/ScrubPartitionIterator.java b/src/java/org/apache/cassandra/io/sstable/format/ScrubPartitionIterator.java new file mode 100644 index 000000000000..f5f8800b161a --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/format/ScrubPartitionIterator.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.io.sstable.format; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Iterator over the partitions of an sstable used for scrubbing. + * + * The difference between this and PartitionIndexIterator is that this only uses information present in the index file + * and does not try to read keys of the data file (for the trie index format), thus key() can be null. + * + * Starts advanced to a position, advance() is to be used to go to next, and iteration completes when dataPosition() == -1. + */ +public interface ScrubPartitionIterator extends Closeable +{ + /** + * Serialized partition key or {@code null} if the iterator reached the end of the index or if the key may not + * be fully retrieved from the index file. + * @return + */ + ByteBuffer key(); + + /** + * Key position in data file or -1 if the iterator reached the end of the index. + */ + long dataPosition(); + + /** + * Move to the next position in the index file. + */ + void advance() throws IOException; + + boolean isExhausted(); + + void close(); +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/io/sstable/format/Version.java b/src/java/org/apache/cassandra/io/sstable/format/Version.java index aa41b149c2a9..4b115cae054f 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/Version.java +++ b/src/java/org/apache/cassandra/io/sstable/format/Version.java @@ -68,6 +68,16 @@ protected Version(SSTableFormat format, String version) public abstract boolean hasAccurateMinMax(); + /** + * If the sstable has improved min/max encoding. + */ + public abstract boolean hasImprovedMinMax(); + + /** + * Records in th stats if the sstable has any partition deletions. + */ + public abstract boolean hasPartitionLevelDeletionsPresenceMarker(); + public String getVersion() { return version; @@ -116,5 +126,14 @@ public int hashCode() return version != null ? version.hashCode() : 0; } + // the fields below are present only in DSE but we do not use them here; though in order to be able to read + // DSE sstables we need to at least skip that data + public abstract boolean hasZeroCopyMetadata(); + + public abstract boolean hasIncrementalNodeSyncMetadata(); + + // TODO TBD + public abstract boolean hasMaxColumnValueLengths(); + public abstract boolean hasOriginatingHostId(); } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java index ff0d7916672c..b28778ba7197 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java @@ -17,13 +17,20 @@ */ package org.apache.cassandra.io.sstable.format.big; +import java.io.IOException; import java.util.Collection; +import java.util.Set; import java.util.UUID; +import com.google.common.collect.ImmutableSet; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.SSTable; +import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.io.util.FileHandle; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; -import org.apache.cassandra.db.RowIndexEntry; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.io.sstable.Descriptor; @@ -31,6 +38,8 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.net.MessagingService; +import static org.apache.cassandra.io.sstable.format.SSTableReaderBuilder.defaultIndexHandleBuilder; + /** * Legacy bigtable format */ @@ -41,11 +50,42 @@ public class BigFormat implements SSTableFormat private static final SSTableReader.Factory readerFactory = new ReaderFactory(); private static final SSTableWriter.Factory writerFactory = new WriterFactory(); + private final static Set REQUIRED_COMPONENTS = ImmutableSet.of(Component.DATA, + Component.PRIMARY_INDEX, + Component.STATS); + + private final static Set SUPPORTED_COMPONENTS = ImmutableSet.of(Component.DATA, + Component.PRIMARY_INDEX, + Component.FILTER, + Component.COMPRESSION_INFO, + Component.STATS, + Component.DIGEST, + Component.CRC, + Component.SUMMARY, + Component.TOC); + + private final static Set STREAMING_COMPONENTS = ImmutableSet.of(Component.DATA, + Component.PRIMARY_INDEX, + Component.SUMMARY, + Component.STATS, + Component.COMPRESSION_INFO, + Component.FILTER, + Component.DIGEST, + Component.CRC); + + private final static Set PRIMARY_INDEX_COMPONENTS = ImmutableSet.of(Component.PRIMARY_INDEX); + private BigFormat() { } + @Override + public Type getType() + { + return Type.BIG; + } + @Override public Version getLatestVersion() { @@ -71,13 +111,40 @@ public SSTableReader.Factory getReaderFactory() } @Override - public RowIndexEntry.IndexSerializer getIndexSerializer(TableMetadata metadata, Version version, SerializationHeader header) + public Set requiredComponents() + { + return REQUIRED_COMPONENTS; + } + + @Override + public Set supportedComponents() { - return new RowIndexEntry.Serializer(version, header); + return SUPPORTED_COMPONENTS; + } + + @Override + public Set streamingComponents() + { + return STREAMING_COMPONENTS; + } + + @Override + public Set primaryIndexComponents() + { + return PRIMARY_INDEX_COMPONENTS; } static class WriterFactory extends SSTableWriter.Factory { + @Override + public long estimateSize(SSTableWriter.SSTableSizeParameters parameters) + { + return (long) ((parameters.partitionCount() // index entries + + parameters.partitionCount() // keys in data file + + parameters.dataSize()) // data + * 1.2); // bloom filter and row index overhead + } + @Override public SSTableWriter open(Descriptor descriptor, long keyCount, @@ -88,19 +155,31 @@ public SSTableWriter open(Descriptor descriptor, MetadataCollector metadataCollector, SerializationHeader header, Collection observers, - LifecycleNewTracker lifecycleNewTracker) + LifecycleNewTracker lifecycleNewTracker, + Set indexComponents) { SSTable.validateRepairedMetadata(repairedAt, pendingRepair, isTransient); - return new BigTableWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers, lifecycleNewTracker); + return new BigTableWriter(descriptor, keyCount, repairedAt, pendingRepair, isTransient, metadata, metadataCollector, header, observers, lifecycleNewTracker, indexComponents); } } - static class ReaderFactory extends SSTableReader.Factory + static class ReaderFactory extends SSTableReader.AbstractBigTableReaderFactory { @Override - public SSTableReader open(SSTableReaderBuilder builder) + public PartitionIndexIterator indexIterator(Descriptor descriptor, TableMetadata metadata) { - return new BigTableReader(builder); + try (FileHandle iFile = defaultIndexHandleBuilder(descriptor, Component.PRIMARY_INDEX).complete()) { + SerializationHeader.Component headerComponent = (SerializationHeader.Component) + descriptor.getMetadataSerializer() + .deserialize(descriptor, MetadataType.HEADER); + SerializationHeader header = headerComponent.toHeader(metadata); + BigTableRowIndexEntry.Serializer serializer = new BigTableRowIndexEntry.Serializer(descriptor.version, header); + return BigTablePartitionIndexIterator.create(iFile, serializer); + } + catch (IOException ex) + { + throw new RuntimeException(ex); + } } } @@ -133,6 +212,8 @@ static class BigVersion extends Version private final boolean hasCommitLogIntervals; private final boolean hasAccurateMinMax; private final boolean hasOriginatingHostId; + private final boolean hasImprovedMinMax; + private final boolean hasPartitionLevelDeletionPresenceMarker; public final boolean hasMaxCompressedLength; private final boolean hasPendingRepair; private final boolean hasMetadataChecksum; @@ -155,6 +236,8 @@ static class BigVersion extends Version hasCommitLogIntervals = version.compareTo("mc") >= 0; hasAccurateMinMax = version.compareTo("md") >= 0; hasOriginatingHostId = version.matches("(m[e-z])|(n[b-z])"); + hasImprovedMinMax = false; + hasPartitionLevelDeletionPresenceMarker = false; hasMaxCompressedLength = version.compareTo("na") >= 0; hasPendingRepair = version.compareTo("na") >= 0; hasIsTransient = version.compareTo("na") >= 0; @@ -180,6 +263,7 @@ public boolean hasCommitLogIntervals() return hasCommitLogIntervals; } + @Override public boolean hasPendingRepair() { return hasPendingRepair; @@ -209,6 +293,19 @@ public boolean hasAccurateMinMax() return hasAccurateMinMax; } + @Override + public boolean hasImprovedMinMax() + { + return hasImprovedMinMax; + } + + @Override + public boolean hasPartitionLevelDeletionsPresenceMarker() + { + return hasPartitionLevelDeletionPresenceMarker; + } + + @Override public boolean isCompatible() { return version.compareTo(earliest_supported_version) >= 0 && version.charAt(0) <= current_version.charAt(0); @@ -236,5 +333,23 @@ public boolean hasOldBfFormat() { return hasOldBfFormat; } + + @Override + public boolean hasZeroCopyMetadata() + { + return false; + } + + @Override + public boolean hasIncrementalNodeSyncMetadata() + { + return false; + } + + @Override + public boolean hasMaxColumnValueLengths() + { + return false; + } } } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTablePartitionIndexIterator.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTablePartitionIndexIterator.java new file mode 100644 index 000000000000..180f69073ef7 --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTablePartitionIndexIterator.java @@ -0,0 +1,192 @@ +/* + * 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.sstable.format.big; + +import java.io.IOException; +import java.nio.ByteBuffer; +import javax.annotation.concurrent.NotThreadSafe; + +import org.apache.cassandra.io.sstable.format.PartitionIndexIterator; +import org.apache.cassandra.io.sstable.format.big.BigTableRowIndexEntry.IndexSerializer; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.utils.ByteBufferUtil; + +@NotThreadSafe +// TODO STAR-247: implement unit test +public class BigTablePartitionIndexIterator implements PartitionIndexIterator +{ + private final FileHandle indexFile; + private final RandomAccessReader reader; + private final IndexSerializer rowIndexEntrySerializer; + private final long initialPosition; + + private ByteBuffer key; + private long dataPosition; + private long keyPosition; + + private BigTablePartitionIndexIterator(FileHandle indexFile, + RandomAccessReader reader, + IndexSerializer rowIndexEntrySerializer) + { + this.indexFile = indexFile; + this.reader = reader; + this.rowIndexEntrySerializer = rowIndexEntrySerializer; + this.initialPosition = reader.getFilePointer(); + } + + public static BigTablePartitionIndexIterator create(RandomAccessReader reader, IndexSerializer serializer) + throws IOException + { + BigTablePartitionIndexIterator iterator = new BigTablePartitionIndexIterator(null, reader, serializer); + try + { + iterator.advance(); + return iterator; + } + catch (IOException | RuntimeException ex) + { + iterator.close(); + throw ex; + } + } + + @SuppressWarnings({ "resource" }) + public static BigTablePartitionIndexIterator create(FileHandle indexFile, IndexSerializer serializer) + throws IOException + { + FileHandle iFile = null; + RandomAccessReader reader = null; + BigTablePartitionIndexIterator iterator = null; + try + { + iFile = indexFile.sharedCopy(); + reader = iFile.createReader(); + iterator = new BigTablePartitionIndexIterator(iFile, reader, serializer); + iterator.advance(); + return iterator; + } + catch (IOException | RuntimeException ex) + { + if (iterator != null) + { + iterator.close(); + } + else + { + FileUtils.closeQuietly(reader); + FileUtils.closeQuietly(iFile); + } + throw ex; + } + } + + @Override + public void close() + { + key = null; + dataPosition = -1; + keyPosition = -1; + FileUtils.closeQuietly(reader); + FileUtils.closeQuietly(indexFile); + } + + @Override + public boolean advance() throws IOException + { + if (!reader.isEOF()) + { + keyPosition = reader.getFilePointer(); + key = ByteBufferUtil.readWithShortLength(reader); + dataPosition = rowIndexEntrySerializer.deserializePositionAndSkip(reader); + return true; + } + else + { + keyPosition = -1; + dataPosition = -1; + key = null; + return false; + } + } + + @Override + public boolean isExhausted() + { + return key == null && dataPosition < 0; + } + + @Override + public ByteBuffer key() + { + return key; + } + + @Override + public long keyPosition() + { + return keyPosition; + } + + @Override + public long dataPosition() + { + return dataPosition; + } + + @Override + public long indexPosition() + { + return reader.getFilePointer(); + } + + @Override + public void indexPosition(long position) throws IOException + { + if (position > indexLength()) + throw new IndexOutOfBoundsException("The requested position exceeds the index length"); + reader.seek(position); + key = null; + keyPosition = 0; + dataPosition = 0; + advance(); + } + + @Override + public long indexLength() + { + return reader.length(); + } + + @Override + public void reset() throws IOException + { + reader.seek(initialPosition); + key = null; + keyPosition = 0; + dataPosition = 0; + advance(); + } + + @Override + public String toString() + { + return String.format("BigTable-PartitionIndexIterator(%s)", indexFile.path()); + } +} diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java index f60c9dfee473..3a8b0626640e 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java @@ -17,17 +17,21 @@ */ package org.apache.cassandra.io.sstable.format.big; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.*; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; -import org.apache.cassandra.io.sstable.format.SSTableReaderBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.*; -import org.apache.cassandra.db.columniterator.SSTableIterator; -import org.apache.cassandra.db.columniterator.SSTableReversedIterator; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.Slices; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.rows.Rows; import org.apache.cassandra.db.rows.UnfilteredRowIterator; @@ -35,12 +39,19 @@ import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.io.sstable.*; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.CorruptSSTableException; +import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.SSTableIdentityIterator; +import org.apache.cassandra.io.sstable.format.PartitionIndexIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableReaderBuilder; import org.apache.cassandra.io.sstable.format.SSTableReadsListener; import org.apache.cassandra.io.sstable.format.SSTableReadsListener.SelectionReason; import org.apache.cassandra.io.sstable.format.SSTableReadsListener.SkippingReason; +import org.apache.cassandra.io.sstable.format.ScrubPartitionIterator; import org.apache.cassandra.io.util.FileDataInput; +import org.apache.cassandra.io.util.RandomAccessReader; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.ByteBufferUtil; @@ -52,9 +63,29 @@ public class BigTableReader extends SSTableReader { private static final Logger logger = LoggerFactory.getLogger(BigTableReader.class); - BigTableReader(SSTableReaderBuilder builder) + protected final BigTableRowIndexEntry.IndexSerializer rowIndexEntrySerializer; + + @Override + public boolean hasIndex() + { + return new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).exists(); + } + + public BigTableReader(SSTableReaderBuilder builder) { super(builder); + this.rowIndexEntrySerializer = new BigTableRowIndexEntry.Serializer(descriptor.version, header); + } + + @Override + public void setup(boolean trackHotness) { + tidy.setup(this, trackHotness, Arrays.asList(bf, indexSummary, dfile, ifile)); + super.setup(trackHotness); + } + @Override + public PartitionIndexIterator allKeysIterator() throws IOException + { + return BigTablePartitionIndexIterator.create(getIndexFile(), rowIndexEntrySerializer); } public UnfilteredRowIterator iterator(DecoratedKey key, @@ -63,12 +94,12 @@ public UnfilteredRowIterator iterator(DecoratedKey key, boolean reversed, SSTableReadsListener listener) { - RowIndexEntry rie = getPosition(key, SSTableReader.Operator.EQ, listener); + BigTableRowIndexEntry rie = getPosition(key, SSTableReader.Operator.EQ, true, false, listener); return iterator(null, key, rie, slices, selectedColumns, reversed); } @SuppressWarnings("resource") - public UnfilteredRowIterator iterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, Slices slices, ColumnFilter selectedColumns, boolean reversed) + public UnfilteredRowIterator iterator(FileDataInput file, DecoratedKey key, BigTableRowIndexEntry indexEntry, Slices slices, ColumnFilter selectedColumns, boolean reversed) { if (indexEntry == null) return UnfilteredRowIterators.noRowsIterator(metadata(), key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, reversed); @@ -121,8 +152,11 @@ public ISSTableScanner getScanner(Collection> ranges) @SuppressWarnings("resource") // caller to close @Override - public UnfilteredRowIterator simpleIterator(FileDataInput dfile, DecoratedKey key, RowIndexEntry position, boolean tombstoneOnly) + public UnfilteredRowIterator simpleIterator(FileDataInput dfile, DecoratedKey key, boolean tombstoneOnly) { + BigTableRowIndexEntry position = getPosition(key, SSTableReader.Operator.EQ, true, false, SSTableReadsListener.NOOP_LISTENER); + if (position == null) + return null; return SSTableIdentityIterator.create(this, dfile, position, key, tombstoneOnly); } @@ -132,12 +166,13 @@ public UnfilteredRowIterator simpleIterator(FileDataInput dfile, DecoratedKey ke * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins. * @param updateCacheAndStats true if updating stats and cache * @return The index entry corresponding to the key, or null if the key is not present + * TODO @param permitMatchPastLast is always set to false, we should remove it */ - protected RowIndexEntry getPosition(PartitionPosition key, - Operator op, - boolean updateCacheAndStats, - boolean permitMatchPastLast, - SSTableReadsListener listener) + protected BigTableRowIndexEntry getPosition(PartitionPosition key, + Operator op, + boolean updateCacheAndStats, + boolean permitMatchPastLast, + SSTableReadsListener listener) { if (op == Operator.EQ) { @@ -146,7 +181,7 @@ protected RowIndexEntry getPosition(PartitionPosition key, { listener.onSSTableSkipped(this, SkippingReason.BLOOM_FILTER); Tracing.trace("Bloom filter allows skipping sstable {}", descriptor.generation); - bloomFilterTracker.addTrueNegative(); + getBloomFilterTracker().addTrueNegative(); return null; } } @@ -155,7 +190,7 @@ protected RowIndexEntry getPosition(PartitionPosition key, if ((op == Operator.EQ || op == Operator.GE) && (key instanceof DecoratedKey)) { DecoratedKey decoratedKey = (DecoratedKey) key; - RowIndexEntry cachedPosition = getCachedPosition(decoratedKey, updateCacheAndStats); + BigTableRowIndexEntry cachedPosition = getCachedPosition(decoratedKey, updateCacheAndStats); if (cachedPosition != null) { listener.onSSTableSelected(this, cachedPosition, SelectionReason.KEY_CACHE_HIT); @@ -186,7 +221,7 @@ protected RowIndexEntry getPosition(PartitionPosition key, if (skip) { if (op == Operator.EQ && updateCacheAndStats) - bloomFilterTracker.addFalsePositive(); + getBloomFilterTracker().addFalsePositive(); listener.onSSTableSkipped(this, SkippingReason.MIN_MAX_KEYS); Tracing.trace("Check against min and max keys allows skipping sstable {}", descriptor.generation); return null; @@ -244,7 +279,7 @@ protected RowIndexEntry getPosition(PartitionPosition key, if (opSatisfied) { // read data position from index entry - RowIndexEntry indexEntry = rowIndexEntrySerializer.deserialize(in); + BigTableRowIndexEntry indexEntry = rowIndexEntrySerializer.deserialize(in); if (exactMatch && updateCacheAndStats) { assert key instanceof DecoratedKey; // key can be == to the index key only if it's a true row key @@ -265,13 +300,13 @@ protected RowIndexEntry getPosition(PartitionPosition key, cacheKey(decoratedKey, indexEntry); } if (op == Operator.EQ && updateCacheAndStats) - bloomFilterTracker.addTruePositive(); + getBloomFilterTracker().addTruePositive(); listener.onSSTableSelected(this, indexEntry, SelectionReason.INDEX_ENTRY_FOUND); Tracing.trace("Partition index with {} entries found for sstable {}", indexEntry.columnsIndexCount(), descriptor.generation); return indexEntry; } - RowIndexEntry.Serializer.skip(in, descriptor.version); + BigTableRowIndexEntry.Serializer.skip(in, descriptor.version); } } catch (IOException e) @@ -281,11 +316,31 @@ protected RowIndexEntry getPosition(PartitionPosition key, } if (op == SSTableReader.Operator.EQ && updateCacheAndStats) - bloomFilterTracker.addFalsePositive(); + getBloomFilterTracker().addFalsePositive(); listener.onSSTableSkipped(this, SkippingReason.INDEX_ENTRY_NOT_FOUND); Tracing.trace("Partition index lookup complete (bloom filter false positive) for sstable {}", descriptor.generation); return null; } + @Override + public DecoratedKey keyAt(FileDataInput reader) throws IOException + { + if (reader.isEOF()) return null; + + return decorateKey(ByteBufferUtil.readWithShortLength(reader)); + } + @Override + public RandomAccessReader openKeyComponentReader() + { + return openIndexReader(); + } + + @Override + public ScrubPartitionIterator scrubPartitionsIterator() throws IOException + { + if (ifile == null) + return null; + return new ScrubIterator(ifile, rowIndexEntrySerializer); + } } diff --git a/src/java/org/apache/cassandra/db/RowIndexEntry.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableRowIndexEntry.java similarity index 91% rename from src/java/org/apache/cassandra/db/RowIndexEntry.java rename to src/java/org/apache/cassandra/io/sstable/format/big/BigTableRowIndexEntry.java index 215768bc8d3f..66b0b4509b5f 100644 --- a/src/java/org/apache/cassandra/db/RowIndexEntry.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableRowIndexEntry.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.db; +package org.apache.cassandra.io.sstable.format.big; import java.io.IOException; import java.nio.ByteBuffer; @@ -24,8 +24,11 @@ import com.codahale.metrics.Histogram; import org.apache.cassandra.cache.IMeasurableMemory; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.ISerializer; -import org.apache.cassandra.io.sstable.IndexInfo; +import org.apache.cassandra.io.sstable.format.RowIndexEntry; import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputBuffer; @@ -103,7 +106,7 @@ * This results in these classes: *

    *