From c0ab81aa774541f58c33ba2c9ef4664439a2d535 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 3 Jul 2018 12:07:11 -0700 Subject: [PATCH 01/64] Optimize Streaming --- .../org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/DatabaseDescriptor.java | 14 + .../cassandra/db/ColumnFamilyStore.java | 2 +- .../cassandra/db/lifecycle/LogFile.java | 2 +- .../cassandra/db/lifecycle/LogReplicaSet.java | 3 +- .../streaming/CassandraBlockStreamReader.java | 172 +++++++++++++ .../streaming/CassandraBlockStreamWriter.java | 115 +++++++++ .../db/streaming/CassandraIncomingFile.java | 23 +- .../db/streaming/CassandraOutgoingFile.java | 106 +++++++- .../db/streaming/CassandraStreamHeader.java | 99 ++++++- .../db/streaming/CassandraStreamManager.java | 3 +- .../db/streaming/CassandraStreamReader.java | 3 +- .../db/streaming/CassandraStreamWriter.java | 3 +- .../cassandra/db/streaming/ComponentInfo.java | 97 +++++++ .../cassandra/db/streaming/IStreamReader.java | 32 +++ .../cassandra/db/streaming/IStreamWriter.java | 31 +++ .../apache/cassandra/dht/RangeStreamer.java | 1 + .../cassandra/io/sstable/Component.java | 51 ++-- .../cassandra/io/sstable/SSTableLoader.java | 2 +- .../format/big/BigTableBlockWriter.java | 222 ++++++++++++++++ .../io/util/BufferedDataOutputStreamPlus.java | 2 +- .../cassandra/io/util/DataOutputPlus.java | 2 +- .../io/util/RebufferingInputStream.java | 10 +- .../cassandra/io/util/SequentialWriter.java | 17 +- .../util/UnbufferedDataOutputStreamPlus.java | 2 +- .../net/async/ByteBufDataOutputPlus.java | 3 +- .../async/ByteBufDataOutputStreamPlus.java | 50 +++- .../async/NonClosingDefaultFileRegion.java | 49 ++++ .../RebufferingByteBufDataInputPlus.java | 53 ++++ .../cassandra/service/StorageService.java | 6 +- .../streaming/StreamCoordinator.java | 3 +- .../streaming/StreamReceiveTask.java | 5 +- .../streaming/StreamResultFuture.java | 13 +- .../cassandra/streaming/StreamSession.java | 6 +- .../async/NettyStreamingMessageSender.java | 2 +- .../async/StreamingInboundHandler.java | 2 +- .../streaming/messages/StreamInitMessage.java | 7 +- .../apache/cassandra/utils/Collectors3.java | 54 ++++ .../ZeroCopyStreamingBenchmark.java | 243 ++++++++++++++++++ .../CassandraBlockStreamWriterTest.java | 219 ++++++++++++++++ .../streaming/CassandraOutgoingFileTest.java | 101 ++++++++ .../streaming/CassandraStreamHeaderTest.java | 23 ++ .../db/streaming/ComponentInfoTest.java | 62 +++++ .../io/sstable/BigTableWriterTest.java | 5 +- .../cassandra/io/sstable/ComponentTest.java | 41 +++ .../io/sstable/LegacySSTableTest.java | 4 +- .../format/big/BigTableBlockWriterTest.java | 196 ++++++++++++++ .../serializers/SerializationUtils.java | 1 - .../streaming/StreamTransferTaskTest.java | 4 +- .../streaming/StreamingTransferTest.java | 9 +- 50 files changed, 2090 insertions(+), 86 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java create mode 100644 src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java create mode 100644 src/java/org/apache/cassandra/db/streaming/ComponentInfo.java create mode 100644 src/java/org/apache/cassandra/db/streaming/IStreamReader.java create mode 100644 src/java/org/apache/cassandra/db/streaming/IStreamWriter.java create mode 100644 src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java create mode 100644 src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java create mode 100644 src/java/org/apache/cassandra/utils/Collectors3.java create mode 100644 test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java create mode 100644 test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java create mode 100644 test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java create mode 100644 test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java create mode 100644 test/unit/org/apache/cassandra/io/sstable/ComponentTest.java create mode 100644 test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index d9250bb18121..30d8c1e7b56d 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -380,6 +380,7 @@ public class Config public int block_for_peers_timeout_in_secs = 10; public volatile boolean automatic_sstable_upgrade = false; public volatile int max_concurrent_automatic_sstable_upgrades = 1; + public boolean streaming_full_sstable_transfers_enabled = true; public volatile AuditLogOptions audit_logging_options = new AuditLogOptions(); diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 91ee63a7821b..97d07b7a0732 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2260,6 +2260,20 @@ public static int getStreamingConnectionsPerHost() return conf.streaming_connections_per_host; } + public static boolean isFullSSTableTransfersEnabled() + { + if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) + { + logger.debug("Internode encryption enabled. Disabling zero copy ssl table transfers for streaming."); + return false; + } + else + { + return Boolean.valueOf(System.getProperty(Config.PROPERTY_PREFIX + "streaming.enableFullSSTableTransfers", + Boolean.toString(conf.streaming_full_sstable_transfers_enabled))); + } + } + public static String getLocalDataCenter() { return localDC; diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 9c4921e5734d..f03ffe628ea0 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -791,7 +791,7 @@ public Descriptor newSSTableDescriptor(File directory, SSTableFormat.Type format return newSSTableDescriptor(directory, format.info.getLatestVersion(), format); } - private Descriptor newSSTableDescriptor(File directory, Version version, SSTableFormat.Type format) + public Descriptor newSSTableDescriptor(File directory, Version version, SSTableFormat.Type format) { return new Descriptor(version, directory, diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogFile.java b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java index af6f4352a484..98be0a0816df 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogFile.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogFile.java @@ -66,7 +66,7 @@ final class LogFile implements AutoCloseable private final LogReplicaSet replicas = new LogReplicaSet(); // The transaction records, this set must be ORDER PRESERVING - private final LinkedHashSet records = new LinkedHashSet<>(); + private final Set records = Collections.synchronizedSet(new LinkedHashSet<>()); // TODO: Hack until we fix CASSANDRA-14554 // The type of the transaction private final OperationType type; diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java index 65be285b050e..0162f5ab0772 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; import java.util.stream.Collectors; @@ -45,7 +46,7 @@ public class LogReplicaSet implements AutoCloseable { private static final Logger logger = LoggerFactory.getLogger(LogReplicaSet.class); - private final Map replicasByFile = new LinkedHashMap<>(); + private final Map replicasByFile = new ConcurrentHashMap<>(); // TODO: Hack until we fix CASSANDRA-14554 private Collection replicas() { diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java new file mode 100644 index 000000000000..82ac4780026c --- /dev/null +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.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.db.streaming; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Set; + +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.SerializationHeader; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.format.SSTableFormat; +import org.apache.cassandra.io.sstable.format.Version; +import org.apache.cassandra.io.sstable.format.big.BigTableBlockWriter; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.streaming.ProgressInfo; +import org.apache.cassandra.streaming.StreamReceiver; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.messages.StreamMessageHeader; +import org.apache.cassandra.utils.Collectors3; +import org.apache.cassandra.utils.FBUtilities; + +/** + * CassandraBlockStreamReader reads SSTable off the wire and writes it to disk. + */ +public class CassandraBlockStreamReader implements IStreamReader +{ + private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamReader.class); + protected final TableId tableId; + protected final StreamSession session; + protected final int sstableLevel; + protected final SerializationHeader.Component header; + protected final int fileSeqNum; + private final List components; + private final SSTableFormat.Type format; + private final Version version; + + public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHeader streamHeader, StreamSession session) + { + if (session.getPendingRepair() != null) + { + // we should only ever be streaming pending repair + // sstables if the session has a pending repair id + if (!session.getPendingRepair().equals(header.pendingRepair)) + throw new IllegalStateException(String.format("Stream Session & SSTable ({}) pendingRepair UUID mismatch.", + header.tableId)); + } + this.session = session; + this.tableId = header.tableId; + this.components = streamHeader.components; + this.sstableLevel = streamHeader.sstableLevel; + this.header = streamHeader.header; + this.format = streamHeader.format; + this.fileSeqNum = header.sequenceNumber; + this.version = streamHeader.version; + } + + /** + * @param inputPlus where this reads data from + * @return SSTable transferred + * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails. + */ + + @SuppressWarnings("resource") // input needs to remain open, streams on top of it can't be closed + @Override + public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException + { + long totalSize = totalSize(); + + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); + if (cfs == null) + { + // schema was dropped during streaming + throw new IOException("CF " + tableId + " was dropped during streaming"); + } + + logger.debug("[Stream #{}] Start receiving file #{} from {}, size = {}, ks = '{}', table = '{}'.", + session.planId(), fileSeqNum, session.peer, totalSize, cfs.keyspace.getName(), cfs.getTableName()); + + BigTableBlockWriter writer = null; + + try + { + Set componentsToWrite = components.stream() + .map(p -> Component.parse(p.type.repr)) + .collect(Collectors3.toImmutableSet()); + + writer = createWriter(cfs, totalSize, componentsToWrite); + long bytesRead = 0; + for (ComponentInfo info : components) + { + logger.debug("[Stream #{}] About to receive file {} from {} readBytes = {}, componentSize = {}, totalSize = {}", + session.planId(), info.type, session.peer, FBUtilities.prettyPrintMemory(bytesRead), + FBUtilities.prettyPrintMemory(info.length), FBUtilities.prettyPrintMemory(totalSize)); + writer.writeComponent(info.type, inputPlus, info.length); + session.progress(writer.descriptor.filenameFor(Component.parse(info.type.repr)), ProgressInfo.Direction.IN, info.length, info.length); + bytesRead += info.length; + logger.debug("[Stream #{}] Finished receiving file {} from {} readBytes = {}, componentSize = {}, totalSize = {}", + session.planId(), info.type, session.peer, FBUtilities.prettyPrintMemory(bytesRead), + FBUtilities.prettyPrintMemory(info.length), FBUtilities.prettyPrintMemory(totalSize)); + } + + return writer; + } + catch (Throwable e) + { + logger.error("[Stream {}] Error while reading from stream on ks='{}' and table='{}'.", + session.planId(), cfs.keyspace.getName(), cfs.getTableName(), e); + if (writer != null) + { + writer.abort(e); + } + throw Throwables.propagate(e); + } + } + + @SuppressWarnings("resource") + protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize, Set componentsToWrite) throws IOException + { + Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize); + if (localDir == null) + throw new IOException(String.format("Insufficient disk space to store %s", FBUtilities.prettyPrintMemory(totalSize))); + + StreamReceiver streamReceiver = session.getAggregator(tableId); + + assert streamReceiver instanceof CassandraStreamReceiver; + + LifecycleTransaction txn = CassandraStreamReceiver.fromReceiver(session.getAggregator(tableId)).getTransaction(); + + // TODO: Is this the correct directory? + File dir = cfs.getDirectories().getDirectoryForNewSSTables(); + Descriptor desc = cfs.newSSTableDescriptor(dir, version, format); + + logger.debug("[Table #{}] {} Components to write - {}", tableId, desc.filenameFor(Component.DATA), componentsToWrite); + BigTableBlockWriter writer = new BigTableBlockWriter(desc, cfs.metadata, txn, componentsToWrite); + return writer; + } + + protected long totalSize() + { + long size = 0; + for (ComponentInfo component : components) + size += component.length; + return size; + } +} diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java new file mode 100644 index 000000000000..59d64954de03 --- /dev/null +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.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.db.streaming; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.DataOutputStreamPlus; +import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; +import org.apache.cassandra.streaming.ProgressInfo; +import org.apache.cassandra.streaming.StreamManager; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.streaming.StreamManager.StreamRateLimiter; + +/** + * CassandraBlockStreamWriter streams the entire SSTable to given channel. + */ +public class CassandraBlockStreamWriter implements IStreamWriter +{ + private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamWriter.class); + + protected final SSTableReader sstable; + protected final List components; + protected final StreamSession session; + + + public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, List components) + { + this.session = session; + this.sstable = sstable; + this.components = components; + } + + /** + * Stream the entire file to given channel. + *

+ * + * @param output where this writes data to + * @throws IOException on any I/O error + */ + @Override + public void write(DataOutputStreamPlus output) throws IOException + { + long totalSize = totalSize(); + logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}", session.planId(), + sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize); + + long progress = 0L; + ByteBufDataOutputStreamPlus byteBufDataOutputStreamPlus = (ByteBufDataOutputStreamPlus) output; + + for (ComponentInfo info : components) + { + FileChannel in = new RandomAccessFile(sstable.descriptor.filenameFor(Component.parse(info.type.repr)), "r").getChannel(); + + // Total Length to transmit for this file + long length = in.size(); + + // tracks write progress + long bytesRead = 0; + logger.debug("[Stream #{}] Block streaming {}.{} gen {} component {} size {}", session.planId(), + sstable.getKeyspaceName(), sstable.getColumnFamilyName(), sstable.descriptor.generation, info.type, length); + + bytesRead += byteBufDataOutputStreamPlus.writeToChannel(in); + progress += bytesRead; + + session.progress(sstable.descriptor.filenameFor(Component.parse(info.type.repr)), ProgressInfo.Direction.OUT, bytesRead, + length); + + logger.debug("[Stream #{}] Finished block streaming {}.{} gen {} component {} to {}, xfered = {}, length = {}, totalSize = {}", + session.planId(), sstable.getKeyspaceName(), sstable.getColumnFamilyName(), + sstable.descriptor.generation, info.type, session.peer, FBUtilities.prettyPrintMemory(bytesRead), + FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); + + byteBufDataOutputStreamPlus.flush(); + + logger.debug("[Stream #{}] Finished block streaming sstable {} to {}, xfered = {}, totalSize = {}", + session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), + FBUtilities.prettyPrintMemory(totalSize)); + + } + } + + protected long totalSize() + { + long size = 0; + for (ComponentInfo component : components) + size += component.length; + return size; + } +} diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java index 16698e583fbd..02fe68dbe12e 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java @@ -22,6 +22,8 @@ import java.util.Objects; import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.io.sstable.SSTableMultiWriter; @@ -45,6 +47,8 @@ public class CassandraIncomingFile implements IncomingStream private volatile SSTableMultiWriter sstable; private volatile long size = -1; + private static final Logger logger = LoggerFactory.getLogger(CassandraIncomingFile.class); + public CassandraIncomingFile(ColumnFamilyStore cfs, StreamSession session, StreamMessageHeader header) { this.cfs = cfs; @@ -56,9 +60,22 @@ public CassandraIncomingFile(ColumnFamilyStore cfs, StreamSession session, Strea public synchronized void read(DataInputPlus in, int version) throws IOException { CassandraStreamHeader streamHeader = CassandraStreamHeader.serializer.deserialize(in, version); - CassandraStreamReader reader = !streamHeader.isCompressed() - ? new CassandraStreamReader(header, streamHeader, session) - : new CompressedCassandraStreamReader(header, streamHeader, session); + + logger.debug("Incoming stream fullStream={} components={}", streamHeader.fullStream, streamHeader.components); + + IStreamReader reader; + + if (streamHeader.fullStream) + { + reader = new CassandraBlockStreamReader(header, streamHeader, session); + } + else + { + reader = !streamHeader.isCompressed() + ? new CassandraStreamReader(header, streamHeader, session) + : new CompressedCassandraStreamReader(header, streamHeader, session); + } + size = streamHeader.size(); sstable = reader.read(in); } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 6ec1f85eb270..f3f89c04d9fb 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -18,17 +18,30 @@ package org.apache.cassandra.db.streaming; +import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Objects; import java.util.UUID; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; - +import com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DecoratedKey; +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.KeyIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataOutputStreamPlus; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.OutgoingStream; import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.streaming.StreamSession; @@ -45,14 +58,28 @@ public class CassandraOutgoingFile implements OutgoingStream private final String filename; private final CassandraStreamHeader header; private final boolean keepSSTableLevel; - - public CassandraOutgoingFile(StreamOperation operation, Ref ref, List sections, long estimatedKeys) + private final List components; + private final boolean isFullyContained; + + public 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 final List> ranges; + private static final Logger logger = LoggerFactory.getLogger(CassandraOutgoingFile.class); + private static final boolean isFullSSTableTransfersEnabled = DatabaseDescriptor.isFullSSTableTransfersEnabled(); + + public CassandraOutgoingFile(StreamOperation operation, Ref ref, + List sections, Collection> ranges, + long estimatedKeys) { Preconditions.checkNotNull(ref.get()); this.ref = ref; this.estimatedKeys = estimatedKeys; this.sections = sections; + this.ranges = ImmutableList.copyOf(ranges); this.filename = ref.get().getFilename(); + this.components = getComponents(ref.get()); + this.isFullyContained = fullyContainedIn(this.ranges, ref.get()); SSTableReader sstable = ref.get(); keepSSTableLevel = operation == StreamOperation.BOOTSTRAP || operation == StreamOperation.REBUILD; @@ -62,7 +89,20 @@ public CassandraOutgoingFile(StreamOperation operation, Ref ref, sections, sstable.compression ? sstable.getCompressionMetadata() : null, keepSSTableLevel ? sstable.getSSTableLevel() : 0, - sstable.header.toComponent()); + sstable.header.toComponent(), components, shouldStreamFullSSTable()); + } + + private static List getComponents(SSTableReader sstable) + { + List result = new ArrayList<>(STREAM_COMPONENTS.size()); + for (Component component : STREAM_COMPONENTS) + { + File file = new File(sstable.descriptor.filenameFor(component)); + if (file.exists()) + result.add(new ComponentInfo(component.type, file.length())); + } + + return result; } public static CassandraOutgoingFile fromStream(OutgoingStream stream) @@ -114,13 +154,63 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) CassandraStreamHeader.serializer.serialize(header, out, version); out.flush(); - CassandraStreamWriter writer = header.compressionInfo == null ? - new CassandraStreamWriter(sstable, header.sections, session) : - new CompressedCassandraStreamWriter(sstable, header.sections, - header.compressionInfo, session); + IStreamWriter writer; + + if (shouldStreamFullSSTable()) + { + writer = new CassandraBlockStreamWriter(sstable, session, components); + } + else + { + writer = (header.compressionInfo == null) ? + new CassandraStreamWriter(sstable, header.sections, session) : + new CompressedCassandraStreamWriter(sstable, header.sections, + header.compressionInfo, session); + } + writer.write(out); } + @VisibleForTesting + public boolean shouldStreamFullSSTable() + { + return isFullSSTableTransfersEnabled && isFullyContained; + } + + private boolean fullyContainedIn(List> requestedRanges, SSTableReader sstable) + { + if (requestedRanges == null) + return false; + try (KeyIterator iter = new KeyIterator(sstable.descriptor, sstable.metadata())) + { + List> ownedRanges = Range.normalize(StorageService.instance.getLocalAndPendingRanges(sstable.getKeyspaceName())); + if (ownedRanges.isEmpty()) + return false; + + while (iter.hasNext()) + { + DecoratedKey key = iter.next(); + // todo: this can be made more efficient by sorting and normalizing the ranges (they might already be?) + // todo: then we can use the fact that the tokens we get from the sstable are always increasing, se we don't need + // todo: to compare all tokens to all ranges. (see Verifier.RangeOwnHelper in current trunk for example) + + boolean foundFlag = false; + for (Range r : requestedRanges) + { + if (r.contains(key.getToken())) + { + foundFlag = true; + break; + } + } + + if (foundFlag == false) + return false; + } + } + return true; + } + @Override public void finish() { diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index 43631b067f3b..9f4a0c80dc24 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -52,10 +52,18 @@ public class CassandraStreamHeader public final int sstableLevel; public final SerializationHeader.Component header; + /* flag indicating whether this is a partial or full sstable transfer */ + public final boolean fullStream; + public final List components; + /* cached size value */ private transient final long size; - private CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionMetadata compressionMetadata, CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header) + public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, + List sections, CompressionMetadata compressionMetadata, + CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header, + List components, + boolean fullStream) { this.version = version; this.format = format; @@ -65,18 +73,38 @@ private CassandraStreamHeader(Version version, SSTableFormat.Type format, long e this.compressionInfo = compressionInfo; this.sstableLevel = sstableLevel; this.header = header; + this.fullStream = fullStream; + this.components = components; this.size = calculateSize(); } - public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionMetadata compressionMetadata, int sstableLevel, SerializationHeader.Component header) + private CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, + List sections, CompressionMetadata compressionMetadata, + CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header) + { + this(version, format, estimatedKeys, sections, compressionMetadata, compressionInfo, sstableLevel, header, null, false); + } + + public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, + List sections, CompressionMetadata compressionMetadata, + int sstableLevel, SerializationHeader.Component header) { this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header); } - public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header) + public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, + List sections, CompressionMetadata compressionMetadata, + int sstableLevel, SerializationHeader.Component header, List components, boolean fullStream) + { + this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header, components, fullStream); + } + + public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, + List sections, CompressionInfo compressionInfo, + int sstableLevel, SerializationHeader.Component header, List components, boolean fullStream) { - this(version, format, estimatedKeys, sections, null, compressionInfo, sstableLevel, header); + this(version, format, estimatedKeys, sections, null, compressionInfo, sstableLevel, header, components, fullStream); } public boolean isCompressed() @@ -95,16 +123,25 @@ public long size() private long calculateSize() { long transferSize = 0; - if (compressionInfo != null) + + if (fullStream) { - // calculate total length of transferring chunks - for (CompressionMetadata.Chunk chunk : compressionInfo.chunks) - transferSize += chunk.length + 4; // 4 bytes for CRC + for (ComponentInfo info : components) + transferSize += info.length; } else { - for (SSTableReader.PartitionPositionBounds section : sections) - transferSize += section.upperPosition - section.lowerPosition; + if (compressionInfo != null) + { + // calculate total length of transferring chunks + for (CompressionMetadata.Chunk chunk : compressionInfo.chunks) + transferSize += chunk.length + 4; // 4 bytes for CRC + } + else + { + for (SSTableReader.PartitionPositionBounds section : sections) + transferSize += section.upperPosition - section.lowerPosition; + } } return transferSize; } @@ -128,6 +165,8 @@ public String toString() ", compressionInfo=" + compressionInfo + ", sstableLevel=" + sstableLevel + ", header=" + header + + ", fullStream=" + fullStream + + ", components=" + components + '}'; } @@ -138,19 +177,20 @@ public boolean equals(Object o) CassandraStreamHeader that = (CassandraStreamHeader) o; return estimatedKeys == that.estimatedKeys && sstableLevel == that.sstableLevel && + fullStream == that.fullStream && Objects.equals(version, that.version) && format == that.format && Objects.equals(sections, that.sections) && Objects.equals(compressionInfo, that.compressionInfo) && - Objects.equals(header, that.header); + Objects.equals(header, that.header) && + Objects.equals(components, that.components); } public int hashCode() { - return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, header); + return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, components, fullStream); } - public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(CassandraStreamHeader header, DataOutputPlus out, int version) throws IOException @@ -168,7 +208,16 @@ public void serialize(CassandraStreamHeader header, DataOutputPlus out, int vers header.calculateCompressionInfo(); CompressionInfo.serializer.serialize(header.compressionInfo, out, version); out.writeInt(header.sstableLevel); + SerializationHeader.serializer.serialize(header.version, header.header, out); + + out.writeBoolean(header.fullStream); + if (header.fullStream) + { + out.writeInt(header.components.size()); + for (ComponentInfo info : header.components) + ComponentInfo.serializer.serialize(info, out, version); + } } public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws IOException @@ -183,9 +232,22 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws I sections.add(new SSTableReader.PartitionPositionBounds(in.readLong(), in.readLong())); CompressionInfo compressionInfo = CompressionInfo.serializer.deserialize(in, version); int sstableLevel = in.readInt(); + SerializationHeader.Component header = SerializationHeader.serializer.deserialize(sstableVersion, in); - return new CassandraStreamHeader(sstableVersion, format, estimatedKeys, sections, compressionInfo, sstableLevel, header); + boolean fullStream = in.readBoolean(); + List components = null; + + if (fullStream) + { + int ncomp = in.readInt(); + components = new ArrayList<>(ncomp); + + for (int i=0; i < ncomp; i++) + components.add(ComponentInfo.serializer.deserialize(in, version)); + } + + return new CassandraStreamHeader(sstableVersion, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, components, fullStream); } public long serializedSize(CassandraStreamHeader header, int version) @@ -206,6 +268,15 @@ public long serializedSize(CassandraStreamHeader header, int version) size += SerializationHeader.serializer.serializedSize(header.version, header.header); + size += TypeSizes.sizeof(header.fullStream); + + if (header.fullStream) + { + size += TypeSizes.sizeof(header.components.size()); + for (ComponentInfo info : header.components) + size += ComponentInfo.serializer.serializedSize(info, version); + } + return size; } }; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java index 673b62c2c035..5d3c51f582b2 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java @@ -152,7 +152,8 @@ else if (pendingRepair == ActiveRepairService.NO_PENDING_REPAIR) ref.release(); continue; } - streams.add(new CassandraOutgoingFile(session.getStreamOperation(), ref, sections, sstable.estimatedKeysForRanges(ranges))); + streams.add(new CassandraOutgoingFile(session.getStreamOperation(), ref, sections, ranges, + sstable.estimatedKeysForRanges(ranges))); } return streams; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java index 3930196baaac..f0d9c4ee6422 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java @@ -53,7 +53,7 @@ /** * CassandraStreamReader reads from stream and writes to SSTable. */ -public class CassandraStreamReader +public class CassandraStreamReader implements IStreamReader { private static final Logger logger = LoggerFactory.getLogger(CassandraStreamReader.class); protected final TableId tableId; @@ -95,6 +95,7 @@ public CassandraStreamReader(StreamMessageHeader header, CassandraStreamHeader s * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails. */ @SuppressWarnings("resource") // input needs to remain open, streams on top of it can't be closed + @Override public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException { long totalSize = totalSize(); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java index c6dd9a91e6d0..9d29b2c2e76d 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java @@ -42,7 +42,7 @@ /** * CassandraStreamWriter writes given section of the SSTable to given channel. */ -public class CassandraStreamWriter +public class CassandraStreamWriter implements IStreamWriter { private static final int DEFAULT_CHUNK_SIZE = 64 * 1024; @@ -69,6 +69,7 @@ public CassandraStreamWriter(SSTableReader sstable, Collection= 0 : "Component length cannot be negative"; + this.type = type; + this.length = length; + } + + @Override + public String toString() + { + return "ComponentInfo{" + + "type=" + type + + ", length=" + length + + '}'; + } + + public boolean equals(Object o) + { + if (this == o) return true; + + if (o == null || getClass() != o.getClass()) return false; + + ComponentInfo that = (ComponentInfo) o; + + return new EqualsBuilder() + .append(length, that.length) + .append(type, that.type) + .isEquals(); + } + + public int hashCode() + { + return new HashCodeBuilder(17, 37) + .append(type) + .append(length) + .toHashCode(); + } + + public static final IVersionedSerializer serializer = new IVersionedSerializer() + { + public void serialize(ComponentInfo info, DataOutputPlus out, int version) throws IOException + { + out.writeByte(info.type.id); + out.writeLong(info.length); + } + + public ComponentInfo deserialize(DataInputPlus in, int version) throws IOException + { + Component.Type type = Component.Type.fromRepresentation(in.readByte()); + long size = in.readLong(); + return new ComponentInfo(type, size); + } + + public long serializedSize(ComponentInfo info, int version) + { + long size = 0; + size += TypeSizes.sizeof(info.type.id); + size += TypeSizes.sizeof(info.length); + return size; + } + }; +} diff --git a/src/java/org/apache/cassandra/db/streaming/IStreamReader.java b/src/java/org/apache/cassandra/db/streaming/IStreamReader.java new file mode 100644 index 000000000000..cf93bc2c4225 --- /dev/null +++ b/src/java/org/apache/cassandra/db/streaming/IStreamReader.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.streaming; + +import java.io.IOException; + +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.util.DataInputPlus; + +/** + * This is the interface is used by the streaming code read a SSTable stream off a channel. + */ +public interface IStreamReader +{ + public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException; +} diff --git a/src/java/org/apache/cassandra/db/streaming/IStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/IStreamWriter.java new file mode 100644 index 000000000000..94b3ab15c7ff --- /dev/null +++ b/src/java/org/apache/cassandra/db/streaming/IStreamWriter.java @@ -0,0 +1,31 @@ +/* + * 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.streaming; + +import java.io.IOException; + +import org.apache.cassandra.io.util.DataOutputStreamPlus; + +/** + * This is the interface is used by the streaming code to stream SSTables to a channel. + */ +public interface IStreamWriter +{ + public void write(DataOutputStreamPlus output) throws IOException; +} diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java index 110fed639c02..2fc6e21db566 100644 --- a/src/java/org/apache/cassandra/dht/RangeStreamer.java +++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java @@ -431,6 +431,7 @@ public StreamResultFuture fetchAsync() // filter out already streamed ranges Set> availableRanges = stateStore.getAvailableRanges(keyspace, StorageService.instance.getTokenMetadata().partitioner); + if (ranges.removeAll(availableRanges)) { logger.info("Some ranges of {} are already available. Skipping streaming those ranges.", availableRanges); diff --git a/src/java/org/apache/cassandra/io/sstable/Component.java b/src/java/org/apache/cassandra/io/sstable/Component.java index 9daac7cdb4c9..40bcaf6a7dfc 100644 --- a/src/java/org/apache/cassandra/io/sstable/Component.java +++ b/src/java/org/apache/cassandra/io/sstable/Component.java @@ -18,9 +18,12 @@ package org.apache.cassandra.io.sstable; import java.util.EnumSet; +import java.util.HashMap; +import java.util.Map; import java.util.regex.Pattern; import com.google.common.base.Objects; +import com.google.common.collect.ImmutableMap; /** * SSTables are made up of multiple components in separate files. Components are @@ -37,35 +40,39 @@ public enum Type { // the base data for an sstable: the remaining components can be regenerated // based on the data component - DATA("Data.db"), + DATA("Data.db", 1), // index of the row keys with pointers to their positions in the data file - PRIMARY_INDEX("Index.db"), + PRIMARY_INDEX("Index.db", 2), // serialized bloom filter for the row keys in the sstable - FILTER("Filter.db"), + FILTER("Filter.db", 3), // file to hold information about uncompressed data length, chunk offsets etc. - COMPRESSION_INFO("CompressionInfo.db"), + COMPRESSION_INFO("CompressionInfo.db", 4), // statistical metadata about the content of the sstable - STATS("Statistics.db"), + STATS("Statistics.db", 5), // holds CRC32 checksum of the data file - DIGEST("Digest.crc32"), + DIGEST("Digest.crc32", 6), // holds the CRC32 for chunks in an a uncompressed file. - CRC("CRC.db"), + CRC("CRC.db", 7), // holds SSTable Index Summary (sampling of Index component) - SUMMARY("Summary.db"), + SUMMARY("Summary.db", 8), // table of contents, stores the list of all components for the sstable - TOC("TOC.txt"), + TOC("TOC.txt", 9), // built-in secondary index (may be multiple per sstable) - SECONDARY_INDEX("SI_.*.db"), + SECONDARY_INDEX("SI_.*.db", 10), // custom component, used by e.g. custom compaction strategy - CUSTOM(null); + CUSTOM(null, 11); - final String repr; - Type(String repr) + public final String repr; + public final byte id; + public static final Map idToType; + + Type(String repr, int id) { this.repr = repr; + this.id = (byte) id; } - static Type fromRepresentation(String repr) + public static Type fromRepresentation(String repr) { for (Type type : TYPES) { @@ -74,6 +81,20 @@ static Type fromRepresentation(String repr) } return CUSTOM; } + + public static Type fromRepresentation(byte id) + { + return idToType.getOrDefault(id, CUSTOM); + } + + static { + Type[] values = Type.values(); + Map result = new HashMap<>(values.length); + for (Type t : values) + if (!t.equals(CUSTOM)) result.put(t.id, t); + + idToType = ImmutableMap.copyOf(result); + } } // singleton components for types that don't need ids @@ -120,7 +141,7 @@ public String name() * @return the component corresponding to {@code name}. Note that this always return a component as an unrecognized * name is parsed into a CUSTOM component. */ - static Component parse(String name) + public static Component parse(String name) { Type type = Type.fromRepresentation(name); diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java index ebc35e71f26b..4ba05337bdf5 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java @@ -131,7 +131,7 @@ protected Collection openSSTables(final Map sstableSections = sstable.getPositionsForRanges(tokenRanges); long estimatedKeys = sstable.estimatedKeysForRanges(tokenRanges); Ref ref = sstable.ref(); - OutgoingStream stream = new CassandraOutgoingFile(StreamOperation.BULK_LOAD, ref, sstableSections, estimatedKeys); + OutgoingStream stream = new CassandraOutgoingFile(StreamOperation.BULK_LOAD, ref, sstableSections, tokenRanges, estimatedKeys); streamingDetails.put(endpoint, stream); } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java new file mode 100644 index 000000000000..53f3f146bdd2 --- /dev/null +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.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.io.sstable.format.big; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.io.FSWriteError; +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.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; +import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadataRef; + +public class BigTableBlockWriter extends SSTable implements SSTableMultiWriter +{ + private final TableMetadataRef metadata; + private final LifecycleTransaction txn; + private volatile SSTableReader finalReader; + private final Map componentWriters; + + private final Logger logger = LoggerFactory.getLogger(BigTableBlockWriter.class); + + private final SequentialWriterOption writerOption = SequentialWriterOption.newBuilder() + .trickleFsync(false) + .bufferSize(2 * 1024 * 1024) + .bufferType(BufferType.OFF_HEAP) + .build(); + public static final ImmutableSet supportedComponents = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.STATS, + Component.COMPRESSION_INFO, Component.FILTER, Component.SUMMARY, + Component.DIGEST, Component.CRC); + + public BigTableBlockWriter(Descriptor descriptor, + TableMetadataRef metadata, + LifecycleTransaction txn, + final Set components) + { + super(descriptor, ImmutableSet.copyOf(components), metadata, + DatabaseDescriptor.getDiskOptimizationStrategy()); + txn.trackNew(this); + this.metadata = metadata; + this.txn = txn; + this.componentWriters = new HashMap<>(components.size()); + + assert supportedComponents.containsAll(components) : String.format("Unsupported streaming component detected %s", + new HashSet(components).removeAll(supportedComponents)); + + for (Component c : components) + componentWriters.put(c.type, makeWriter(descriptor, c, writerOption)); + } + + private static SequentialWriter makeWriter(Descriptor descriptor, Component component, SequentialWriterOption writerOption) + { + return new SequentialWriter(new File(descriptor.filenameFor(component)), writerOption, false); + } + + private void write(DataInputPlus in, long size, SequentialWriter out) throws FSWriteError + { + final int BUFFER_SIZE = 1 * 1024 * 1024; + long bytesRead = 0; + byte[] buff = new byte[BUFFER_SIZE]; + try + { + while (bytesRead < size) + { + int toRead = (int) Math.min(size - bytesRead, BUFFER_SIZE); + in.readFully(buff, 0, toRead); + int count = Math.min(toRead, BUFFER_SIZE); + out.write(buff, 0, count); + bytesRead += count; + } + out.sync(); + } + catch (IOException e) + { + throw new FSWriteError(e, out.getPath()); + } + } + + @Override + public boolean append(UnfilteredRowIterator partition) + { + throw new UnsupportedOperationException("Operation not supported by BigTableBlockWriter"); + } + + @Override + public Collection finish(long repairedAt, long maxDataAge, boolean openResult) + { + return finish(openResult); + } + + @Override + public Collection finish(boolean openResult) + { + setOpenResult(openResult); + return finished(); + } + + @Override + public Collection finished() + { + if (finalReader == null) + finalReader = SSTableReader.open(descriptor, + components, + metadata); + + return ImmutableList.of(finalReader); + } + + @Override + public SSTableMultiWriter setOpenResult(boolean openResult) + { + return null; + } + + @Override + public long getFilePointer() + { + return 0; + } + + @Override + public TableId getTableId() + { + return metadata.id; + } + + @Override + public Throwable commit(Throwable accumulate) + { + for (SequentialWriter writer : componentWriters.values()) + writer.commit(accumulate); + + return accumulate; + } + + @Override + public Throwable abort(Throwable accumulate) + { + for (SequentialWriter writer : componentWriters.values()) + writer.abort(accumulate); + + return accumulate; + } + + @Override + public void prepareToCommit() + { + for (SequentialWriter writer : componentWriters.values()) + writer.prepareToCommit(); + } + + @Override + public void close() + { + for (SequentialWriter writer : componentWriters.values()) + writer.close(); + } + + public void writeComponent(Component.Type type, DataInputPlus in, long size) + { + logger.info("Writing component {} to {} length {}", type, componentWriters.get(type).getPath(), size); + if (in instanceof RebufferingByteBufDataInputPlus) + write((RebufferingByteBufDataInputPlus) in, size, componentWriters.get(type)); + else + write(in, size, componentWriters.get(type)); + } + + private void write(RebufferingByteBufDataInputPlus in, long size, SequentialWriter writer) + { + logger.info("Block Writing component to {} length {}", writer.getPath(), size); + + try + { + long bytesWritten = in.consumeUntil(writer, size); + + if (bytesWritten != size) + throw new IOException(String.format("Failed to read correct number of bytes from Channel {}", writer)); + } + catch (IOException e) + { + throw new FSWriteError(e, writer.getPath()); + } + } +} diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java index 54122ee3e791..aa0e30717c72 100644 --- a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java @@ -341,7 +341,7 @@ public void close() throws IOException } @Override - public R applyToChannel(Function f) throws IOException + public R applyToChannel(CheckedFunction f) throws IOException { if (strictFlushing) throw new UnsupportedOperationException(); diff --git a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java index a9dbb68e305a..35abb6f29c5a 100644 --- a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java +++ b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java @@ -41,7 +41,7 @@ public interface DataOutputPlus extends DataOutput * Safe way to operate against the underlying channel. Impossible to stash a reference to the channel * and forget to flush */ - R applyToChannel(Function c) throws IOException; + R applyToChannel(CheckedFunction c) throws IOException; default void writeVInt(long i) throws IOException { diff --git a/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java b/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java index 086f5c9d4768..f0a47fa9bbaa 100644 --- a/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java +++ b/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java @@ -26,7 +26,9 @@ import java.nio.ByteOrder; import net.nicoulaj.compilecommand.annotations.DontInline; +import org.apache.cassandra.io.FSError; import org.apache.cassandra.utils.FastByteOperations; +import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.vint.VIntCoding; import com.google.common.base.Preconditions; @@ -85,7 +87,13 @@ public int read(byte[] b, int off, int len) throws IOException int remaining = buffer.limit() - position; if (remaining == 0) { - reBuffer(); + try + { + reBuffer(); + } catch (EOFException e) + { + throw new EOFException("EOF after " + copied + " bytes out of " + len); + } position = buffer.position(); remaining = buffer.limit() - position; if (remaining == 0) diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java index e71f2fa36647..c773d9b1974c 100644 --- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java +++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java @@ -137,12 +137,23 @@ public SequentialWriter(File file) * @param option Writer option */ public SequentialWriter(File file, SequentialWriterOption option) + { + this(file, option, false); + } + + /** + * Create SequentialWriter for given file with specific writer option. + * @param file + * @param option + * @param strictFlushing + */ + public SequentialWriter(File file, SequentialWriterOption option, boolean strictFlushing) { super(openChannel(file), option.allocateBuffer()); - strictFlushing = true; - fchannel = (FileChannel)channel; + this.strictFlushing = strictFlushing; + this.fchannel = (FileChannel)channel; - filePath = file.getAbsolutePath(); + this.filePath = file.getAbsolutePath(); this.option = option; } diff --git a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java index 54b4cb139238..d9ef01064058 100644 --- a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java @@ -378,7 +378,7 @@ public void write(Memory memory, long offset, long length) throws IOException } @Override - public R applyToChannel(Function f) throws IOException + public R applyToChannel(CheckedFunction f) throws IOException { return f.apply(channel); } diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java index 04734658983c..5501c0bf4854 100644 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java +++ b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java @@ -26,6 +26,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufOutputStream; +import org.apache.cassandra.io.util.CheckedFunction; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.io.util.Memory; import org.apache.cassandra.io.util.UnbufferedDataOutputStreamPlus; @@ -82,7 +83,7 @@ public void write(Memory memory, long offset, long length) throws IOException } @Override - public R applyToChannel(Function c) throws IOException + public R applyToChannel(CheckedFunction c) throws IOException { throw new UnsupportedOperationException(); } diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java index 3a544e4e93ee..959655adac9d 100644 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java @@ -18,19 +18,24 @@ package org.apache.cassandra.net.async; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import com.google.common.util.concurrent.Uninterruptibles; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; +import io.netty.channel.DefaultFileRegion; import io.netty.util.concurrent.Future; import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; import org.apache.cassandra.io.util.DataOutputStreamPlus; @@ -49,6 +54,7 @@ public class ByteBufDataOutputStreamPlus extends BufferedDataOutputStreamPlus private final StreamSession session; private final Channel channel; private final int bufferSize; + private final Logger logger = LoggerFactory.getLogger(ByteBufDataOutputStreamPlus.class); /** * Tracks how many bytes we've written to the netty channel. This more or less follows the channel's @@ -70,7 +76,6 @@ private ByteBufDataOutputStreamPlus(StreamSession session, Channel channel, Byte this.channel = channel; this.currentBuf = buffer; this.bufferSize = bufferSize; - channelRateLimiter = new Semaphore(channel.config().getWriteBufferHighWaterMark(), true); } @@ -114,8 +119,9 @@ public ChannelFuture writeToChannel(ByteBuf buf) throws IOException doFlush(buffer.position()); int byteCount = buf.readableBytes(); + if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, byteCount, 5, TimeUnit.MINUTES)) - throw new IOException("outbound channel was not writable"); + throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", byteCount)); // the (possibly naive) assumption that we should always flush after each incoming buf ChannelFuture channelFuture = channel.writeAndFlush(buf); @@ -135,6 +141,42 @@ public ChannelFuture writeToChannel(ByteBuffer buffer) throws IOException return channelFuture; } + /** + * Writes all data in file channel to stream BUFFER_SIZE at a time. + * Closes file channel when done + * + * @param f + * @return number of bytes transferred + * @throws IOException + */ + public long writeToChannel(FileChannel f) throws IOException + { + final long length = f.size(); + long bytesTransferred = 0; + + while (bytesTransferred < length) + { + int toRead = (int) Math.min(bufferSize, length - bytesTransferred); + NonClosingDefaultFileRegion fileRegion = new NonClosingDefaultFileRegion(f, bytesTransferred, toRead); + + if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, toRead, 5, TimeUnit.MINUTES)) + throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", toRead)); + + bytesTransferred += toRead; + final boolean shouldClose = (bytesTransferred == length); // this is the last buffer, can safely close channel + + channel.writeAndFlush(fileRegion).addListener(future -> { + handleBuffer(future, toRead); + + if ((shouldClose || !future.isSuccess()) && f.isOpen()) + f.close(); + }); + logger.trace("{} of {} (toRead {} cs {})", bytesTransferred, length, toRead, f.isOpen()); + } + + return bytesTransferred; + } + @Override protected void doFlush(int count) throws IOException { @@ -145,7 +187,7 @@ protected void doFlush(int count) throws IOException currentBuf.writerIndex(byteCount); if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, byteCount, 2, TimeUnit.MINUTES)) - throw new IOException("outbound channel was not writable"); + throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", byteCount)); channel.writeAndFlush(currentBuf).addListener(future -> handleBuffer(future, byteCount)); currentBuf = channel.alloc().directBuffer(bufferSize, bufferSize); @@ -161,7 +203,7 @@ protected void doFlush(int count) throws IOException private void handleBuffer(Future future, int bytesWritten) { channelRateLimiter.release(bytesWritten); - + logger.trace("bytesWritten {} {} because {}", bytesWritten, (future.isSuccess() == true) ? "Succeeded" : "Failed", future.cause()); if (!future.isSuccess() && channel.isOpen()) session.onError(future.cause()); } diff --git a/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java b/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java new file mode 100644 index 000000000000..9a0c6ab5e44c --- /dev/null +++ b/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net.async; + +import java.io.File; +import java.nio.channels.FileChannel; + +import io.netty.channel.DefaultFileRegion; + +/** + * Netty's DefaultFileRegion closes the underlying FileChannel as soon as + * the refCnt() for the region closes, this is an implementation of the DFR + * that doesn't close the FileChannel. + */ +public class NonClosingDefaultFileRegion extends DefaultFileRegion +{ + + public NonClosingDefaultFileRegion(FileChannel file, long position, long count) + { + super(file, position, count); + } + + public NonClosingDefaultFileRegion(File f, long position, long count) + { + super(f, position, count); + } + + @Override + protected void deallocate() + { + // Overridden to avoid closing the file + } +} diff --git a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java index 1f32aa879999..cd0f6fdc7e2c 100644 --- a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java +++ b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java @@ -31,7 +31,11 @@ import io.netty.buffer.Unpooled; import io.netty.channel.ChannelConfig; import io.netty.util.ReferenceCountUtil; +import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; import org.apache.cassandra.io.util.RebufferingInputStream; +import org.apache.cassandra.io.util.SequentialWriter; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FastByteOperations; public class RebufferingByteBufDataInputPlus extends RebufferingInputStream implements ReadableByteChannel { @@ -249,4 +253,53 @@ public ByteBufAllocator getAllocator() { return channelConfig.getAllocator(); } + + /** + * Consumes bytes in the stream until the given length + * + * @param writer + * @param len + * @return + * @throws IOException + */ + public long consumeUntil(BufferedDataOutputStreamPlus writer, long len) throws IOException + { + long copied = 0; // number of bytes copied + while (copied < len) + { + int position = buffer.position(); + int remaining = buffer.remaining(); + if (remaining == 0) + { + try + { + reBuffer(); + } catch (EOFException e) + { + throw new EOFException("EOF after " + copied + " bytes out of " + len); + } + position = buffer.position(); + remaining = buffer.remaining(); + if (remaining == 0) + return copied == 0 ? -1 : copied; + } + + int toCopy = (int) Math.min(len - copied, remaining); + + ByteBuffer dup = buffer.duplicate(); + + if (toCopy < remaining) + dup.limit(dup.position() + toCopy); + + int result = writer.applyToChannel(c -> c.write(dup)); + + if (result == -1) + throw new IOException(String.format("Failed to write to Channel {}", writer)); + + buffer.position(position + result); + copied += result; + } + + return copied; + } } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 96fd63f08ba2..1872153ec466 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2816,7 +2816,7 @@ private void restoreReplicaCount(InetAddressAndPort endpoint, final InetAddressA } } - StreamPlan stream = new StreamPlan(StreamOperation.RESTORE_REPLICA_COUNT); + StreamPlan stream = new StreamPlan(StreamOperation.RESTORE_REPLICA_COUNT, false); for (String keyspaceName : rangesToFetch.keySet()) { for (Map.Entry>> entry : rangesToFetch.get(keyspaceName)) @@ -4236,7 +4236,7 @@ private void move(Token newToken) throws IOException private class RangeRelocator { - private final StreamPlan streamPlan = new StreamPlan(StreamOperation.RELOCATION); + private final StreamPlan streamPlan = new StreamPlan(StreamOperation.RELOCATION, false); private RangeRelocator(Collection tokens, List keyspaceNames) { @@ -5116,7 +5116,7 @@ private Future streamRanges(Map, Inet sessionsToStreamByKeyspace.put(keyspace, rangesPerEndpoint); } - StreamPlan streamPlan = new StreamPlan(StreamOperation.DECOMMISSION); + StreamPlan streamPlan = new StreamPlan(StreamOperation.DECOMMISSION, false); // Vinculate StreamStateStore to current StreamPlan to update transferred ranges per StreamSession streamPlan.listeners(streamStateStore); diff --git a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java index 34b0bbdc8da6..7eada287b60b 100644 --- a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java +++ b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java @@ -282,7 +282,8 @@ public StreamSession getOrCreateNextSession(InetAddressAndPort peer) // create if (streamSessions.size() < connectionsPerHost) { - StreamSession session = new StreamSession(streamOperation, peer, factory, streamSessions.size(), pendingRepair, previewKind); + StreamSession session = new StreamSession(streamOperation, peer, factory, streamSessions.size(), + pendingRepair, previewKind); streamSessions.put(++lastReturned, session); return session; } diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java index 49beba1f58e9..0a96f4c008fc 100644 --- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java +++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java @@ -49,6 +49,7 @@ public class StreamReceiveTask extends StreamTask private volatile boolean done = false; private int remoteStreamsReceived = 0; + private long bytesReceived = 0; public StreamReceiveTask(StreamSession session, TableId tableId, int totalStreams, long totalSize) { @@ -76,8 +77,10 @@ public synchronized void received(IncomingStream stream) } remoteStreamsReceived++; + bytesReceived += stream.getSize(); Preconditions.checkArgument(tableId.equals(stream.getTableId())); - logger.debug("recevied {} of {} total files", remoteStreamsReceived, totalStreams); + logger.debug("received {} of {} total files {} of total bytes {}", remoteStreamsReceived, totalStreams, + bytesReceived, totalSize); receiver.received(stream); diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java index ef8976dd6398..4de63be907cd 100644 --- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java +++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java @@ -17,10 +17,8 @@ */ package org.apache.cassandra.streaming; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.util.*; +import java.util.Collection; +import java.util.UUID; import java.util.concurrent.ConcurrentLinkedQueue; import com.google.common.util.concurrent.AbstractFuture; @@ -78,7 +76,7 @@ private StreamResultFuture(UUID planId, StreamOperation streamOperation, UUID pe this(planId, streamOperation, new StreamCoordinator(streamOperation, 0, new DefaultConnectionFactory(), false, pendingRepair, previewKind)); } - static StreamResultFuture init(UUID planId, StreamOperation streamOperation, Collection listeners, + public static StreamResultFuture init(UUID planId, StreamOperation streamOperation, Collection listeners, StreamCoordinator coordinator) { StreamResultFuture future = createAndRegister(planId, streamOperation, coordinator); @@ -112,8 +110,9 @@ public static synchronized StreamResultFuture initReceivingSide(int sessionIndex StreamResultFuture future = StreamManager.instance.getReceivingStream(planId); if (future == null) { - logger.info("[Stream #{} ID#{}] Creating new streaming plan for {} from {} channel.remote {} channel.local {} channel.id {}", - planId, sessionIndex, streamOperation.getDescription(), from, channel.remoteAddress(), channel.localAddress(), channel.id()); + logger.info("[Stream #{} ID#{}] Creating new streaming plan for {} from {} channel.remote {} channel.local {}" + + " channel.id {}", planId, sessionIndex, streamOperation.getDescription(), + from, channel.remoteAddress(), channel.localAddress(), channel.id()); // The main reason we create a StreamResultFuture on the receiving side is for JMX exposure. future = new StreamResultFuture(planId, streamOperation, pendingRepair, previewKind); diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java index c56616ed5c65..393cd2438904 100644 --- a/src/java/org/apache/cassandra/streaming/StreamSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamSession.java @@ -202,7 +202,8 @@ public StreamSession(StreamOperation streamOperation, InetAddressAndPort peer, S this.pendingRepair = pendingRepair; this.previewKind = previewKind; - logger.debug("Creating stream session peer={} preferredPeerInetAddressAndPort={}", peer, preferredPeerInetAddressAndPort); + logger.debug("Creating stream session peer={} preferredPeerInetAddressAndPort={}", peer, + preferredPeerInetAddressAndPort); } public UUID planId() @@ -777,7 +778,8 @@ private void flushSSTables(Iterable stores) FBUtilities.waitOnFutures(flushes); } - private synchronized void prepareReceiving(StreamSummary summary) + @VisibleForTesting + public synchronized void prepareReceiving(StreamSummary summary) { failIfFinished(); if (summary.files > 0) diff --git a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java index bff77cf305db..3fa80f515ab0 100644 --- a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java +++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java @@ -322,7 +322,7 @@ public void run() throw new IllegalStateException("channel's transferring state is currently set to true. refusing to start new stream"); // close the DataOutputStreamPlus as we're done with it - but don't close the channel - try (DataOutputStreamPlus outPlus = ByteBufDataOutputStreamPlus.create(session, channel, 1 << 16)) + try (DataOutputStreamPlus outPlus = ByteBufDataOutputStreamPlus.create(session, channel, 1 << 20)) { StreamMessage.serialize(msg, outPlus, protocolVersion, session); channel.flush(); diff --git a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java index 81fe8cd6e451..7c10ef96efdf 100644 --- a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java +++ b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java @@ -62,7 +62,7 @@ public class StreamingInboundHandler extends ChannelInboundHandlerAdapter static final Function DEFAULT_SESSION_PROVIDER = sid -> StreamManager.instance.findSession(sid.from, sid.planId, sid.sessionIndex); private static final int AUTO_READ_LOW_WATER_MARK = 1 << 15; - private static final int AUTO_READ_HIGH_WATER_MARK = 1 << 16; + private static final int AUTO_READ_HIGH_WATER_MARK = 1 << 20; private final InetAddressAndPort remoteAddress; private final int protocolVersion; diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java index fbd3e21a3824..a591a43f5ce1 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java @@ -47,7 +47,8 @@ public class StreamInitMessage extends StreamMessage public final UUID pendingRepair; public final PreviewKind previewKind; - public StreamInitMessage(InetAddressAndPort from, int sessionIndex, UUID planId, StreamOperation streamOperation, UUID pendingRepair, PreviewKind previewKind) + public StreamInitMessage(InetAddressAndPort from, int sessionIndex, UUID planId, StreamOperation streamOperation, + UUID pendingRepair, PreviewKind previewKind) { super(Type.STREAM_INIT); this.from = from; @@ -93,7 +94,8 @@ public StreamInitMessage deserialize(DataInputPlus in, int version, StreamSessio UUID pendingRepair = in.readBoolean() ? UUIDSerializer.serializer.deserialize(in, version) : null; PreviewKind previewKind = PreviewKind.deserialize(in.readInt()); - return new StreamInitMessage(from, sessionIndex, planId, StreamOperation.fromString(description), pendingRepair, previewKind); + return new StreamInitMessage(from, sessionIndex, planId, StreamOperation.fromString(description), + pendingRepair, previewKind); } public long serializedSize(StreamInitMessage message, int version) @@ -108,6 +110,7 @@ public long serializedSize(StreamInitMessage message, int version) size += UUIDSerializer.serializer.serializedSize(message.pendingRepair, MessagingService.current_version); } size += TypeSizes.sizeof(message.previewKind.getSerializationVal()); + return size; } } diff --git a/src/java/org/apache/cassandra/utils/Collectors3.java b/src/java/org/apache/cassandra/utils/Collectors3.java new file mode 100644 index 000000000000..faad7ee94d1b --- /dev/null +++ b/src/java/org/apache/cassandra/utils/Collectors3.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.utils; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collector; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +/** + * Some extra Collector implementations. + * + * Named Collectors3 just in case Guava ever makes a Collectors2 + */ +public class Collectors3 +{ + private static final Collector.Characteristics[] LIST_CHARACTERISTICS = new Collector.Characteristics[] { }; + public static Collector> toImmutableList() + { + return Collector.of(ImmutableList.Builder::new, + ImmutableList.Builder::add, + (l, r) -> l.addAll(r.build()), + ImmutableList.Builder::build, + LIST_CHARACTERISTICS); + } + + private static final Collector.Characteristics[] SET_CHARACTERISTICS = new Collector.Characteristics[] { Collector.Characteristics.UNORDERED }; + public static Collector> toImmutableSet() + { + return Collector.of(ImmutableSet.Builder::new, + ImmutableSet.Builder::add, + (l, r) -> l.addAll(r.build()), + ImmutableSet.Builder::build, + SET_CHARACTERISTICS); + } +} \ No newline at end of file diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java new file mode 100644 index 000000000000..101862be2975 --- /dev/null +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.test.microbench; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.RowUpdateBuilder; +import org.apache.cassandra.db.commitlog.CommitLog; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.streaming.CassandraBlockStreamReader; +import org.apache.cassandra.db.streaming.CassandraBlockStreamWriter; +import org.apache.cassandra.db.streaming.CassandraStreamHeader; +import org.apache.cassandra.db.streaming.ComponentInfo; +import org.apache.cassandra.db.streaming.CompressionInfo; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.async.ByteBufDataInputPlus; +import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; +import org.apache.cassandra.net.async.NonClosingDefaultFileRegion; +import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.schema.CachingParams; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.streaming.DefaultConnectionFactory; +import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.streaming.SessionInfo; +import org.apache.cassandra.streaming.StreamCoordinator; +import org.apache.cassandra.streaming.StreamEventHandler; +import org.apache.cassandra.streaming.StreamOperation; +import org.apache.cassandra.streaming.StreamResultFuture; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.StreamSummary; +import org.apache.cassandra.streaming.messages.IncomingStreamMessage; +import org.apache.cassandra.streaming.messages.StreamMessageHeader; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +import static org.apache.cassandra.db.streaming.CassandraOutgoingFile.STREAM_COMPONENTS; + +@Warmup(iterations = 1, time = 1, timeUnit = TimeUnit.SECONDS) +@Measurement(iterations = 1, time = 1, timeUnit = TimeUnit.SECONDS) +@Fork(value = 1) +@Threads(1) +public class ZeroCopyStreamingBenchmark +{ + @State(Scope.Thread) + public static class BenchmarkState + { + public static final String KEYSPACE = "ZeroCopyStreamingBenchmark"; + public static final String CF_STANDARD = "Standard1"; + public static final String CF_INDEXED = "Indexed1"; + public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval"; + + private static SSTableReader sstable; + private static ColumnFamilyStore store; + private StreamSession session; + + @Setup + public void defineSchemaAndPrepareSSTable() + { + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE, + KeyspaceParams.simple(1), + SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD), + SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true), + SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL) + .minIndexInterval(8) + .maxIndexInterval(256) + .caching(CachingParams.CACHE_NOTHING)); + + Keyspace keyspace = Keyspace.open(KEYSPACE); + store = keyspace.getColumnFamilyStore("Standard1"); + + // insert data and compact to a single sstable + CompactionManager.instance.disableAutoCompaction(); + for (int j = 0; j < 1_000_000; j++) + { + new RowUpdateBuilder(store.metadata(), j, String.valueOf(j)) + .clustering("0") + .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) + .build() + .applyUnsafe(); + } + store.forceBlockingFlush(); + CompactionManager.instance.performMaximal(store, false); + + sstable = store.getLiveSSTables().iterator().next(); + session = setupStreamingSessionForTest(); + } + + @TearDown + public void tearDown() throws IOException + { + SchemaLoader.cleanupAndLeaveDirs(); + CommitLog.instance.stopUnsafe(true); + } + + private StreamSession setupStreamingSessionForTest() + { + StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new DefaultConnectionFactory(), false, null, PreviewKind.NONE); + StreamResultFuture future = StreamResultFuture.init(UUID.randomUUID(), StreamOperation.BOOTSTRAP, Collections.emptyList(), streamCoordinator); + + InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); + streamCoordinator.addSessionInfo(new SessionInfo(peer, 0, peer, Collections.emptyList(), Collections.emptyList(), StreamSession.State.INITIALIZED)); + + StreamSession session = streamCoordinator.getOrCreateNextSession(peer); + session.init(future); + return session; + } + } + + static final int STREAM_SIZE = 80 * 1024 * 1024; + + + @Benchmark + @BenchmarkMode(Mode.Throughput) + public void stream(BenchmarkState state) throws Exception + { + StreamSession session = state.session; + SSTableReader sstable = state.sstable; + InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); + + CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); + + // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed + ByteBuf serializedFile = Unpooled.buffer(STREAM_SIZE); + EmbeddedChannel channel = createMockNettyChannel(serializedFile); + ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); + + writer.write(out); + + session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, serializedFile.readableBytes())); + + CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), getStreamableComponents(sstable), true); + + CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); + + RebufferingByteBufDataInputPlus in = new RebufferingByteBufDataInputPlus(STREAM_SIZE, STREAM_SIZE, channel.config()); + in.append(serializedFile); + SSTableMultiWriter sstableWriter = reader.read(in); + Collection newSstables = sstableWriter.finished(); + } + + private static List getStreamableComponents(SSTableReader sstable) + { + List result = new ArrayList<>(STREAM_COMPONENTS.size()); + for (Component component : STREAM_COMPONENTS) + { + File file = new File(sstable.descriptor.filenameFor(component)); + if (file.exists()) + { + result.add(new ComponentInfo(component.type, file.length())); + } + } + + return result; + } + + private EmbeddedChannel createMockNettyChannel(ByteBuf serializedFile) throws Exception + { + WritableByteChannel wbc = new WritableByteChannel() + { + private boolean isOpen = true; + + public int write(ByteBuffer src) throws IOException + { + int size = src.limit(); + serializedFile.writeBytes(src); + return size; + } + + public boolean isOpen() + { + return isOpen; + } + + public void close() throws IOException + { + isOpen = false; + } + }; + + EmbeddedChannel channel = new EmbeddedChannel(new ChannelOutboundHandlerAdapter() + { + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception + { + ((NonClosingDefaultFileRegion) msg).transferTo(wbc, 0); + super.write(ctx, msg, promise); + } + }); + + channel.config().setWriteBufferHighWaterMark(STREAM_SIZE); // avoid rate limiting + + return channel; + } +} diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java new file mode 100644 index 000000000000..81c4e0596492 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java @@ -0,0 +1,219 @@ +/* + * 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.streaming; + + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import java.util.UUID; + +import org.junit.BeforeClass; +import org.junit.Test; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultFileRegion; +import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.RowUpdateBuilder; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.sstable.SSTableMultiWriter; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.async.ByteBufDataInputPlus; +import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; +import org.apache.cassandra.net.async.NonClosingDefaultFileRegion; +import org.apache.cassandra.schema.CachingParams; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.streaming.DefaultConnectionFactory; +import org.apache.cassandra.streaming.PreviewKind; +import org.apache.cassandra.streaming.SessionInfo; +import org.apache.cassandra.streaming.StreamCoordinator; +import org.apache.cassandra.streaming.StreamEventHandler; +import org.apache.cassandra.streaming.StreamOperation; +import org.apache.cassandra.streaming.StreamResultFuture; +import org.apache.cassandra.streaming.StreamSession; +import org.apache.cassandra.streaming.StreamSummary; +import org.apache.cassandra.streaming.messages.StreamMessageHeader; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.db.streaming.CassandraOutgoingFile.STREAM_COMPONENTS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class CassandraBlockStreamWriterTest +{ + public static final String KEYSPACE = "CassandraBlockStreamWriterTest"; + public static final String CF_STANDARD = "Standard1"; + public static final String CF_INDEXED = "Indexed1"; + public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval"; + + private static SSTableReader sstable; + private static ColumnFamilyStore store; + + @BeforeClass + public static void defineSchemaAndPrepareSSTable() + { + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE, + KeyspaceParams.simple(1), + SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD), + SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true), + SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL) + .minIndexInterval(8) + .maxIndexInterval(256) + .caching(CachingParams.CACHE_NOTHING)); + + Keyspace keyspace = Keyspace.open(KEYSPACE); + store = keyspace.getColumnFamilyStore("Standard1"); + + // insert data and compact to a single sstable + CompactionManager.instance.disableAutoCompaction(); + for (int j = 0; j < 10; j++) + { + new RowUpdateBuilder(store.metadata(), j, String.valueOf(j)) + .clustering("0") + .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) + .build() + .applyUnsafe(); + } + store.forceBlockingFlush(); + CompactionManager.instance.performMaximal(store, false); + + sstable = store.getLiveSSTables().iterator().next(); + } + + @Test + public void testBlockWriterOverWire() throws IOException + { + StreamSession session = setupStreamingSessionForTest(); + + CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); + + EmbeddedChannel channel = new EmbeddedChannel(); + ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); + writer.write(out); + + Queue msgs = channel.outboundMessages(); + + assertTrue(msgs.peek() instanceof DefaultFileRegion); + } + + @Test + public void testBlockReadingAndWritingOverWire() throws Exception + { + StreamSession session = setupStreamingSessionForTest(); + InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); + + CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); + + // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed + ByteBuf serializedFile = Unpooled.buffer(8192); + EmbeddedChannel channel = createMockNettyChannel(serializedFile); + ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); + + writer.write(out); + + session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104)); + + CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), getStreamableComponents(sstable), true); + + CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); + + SSTableMultiWriter sstableWriter = reader.read(new ByteBufDataInputPlus(serializedFile)); + Collection newSstables = sstableWriter.finished(); + + assertEquals(1, newSstables.size()); + } + + private EmbeddedChannel createMockNettyChannel(ByteBuf serializedFile) throws Exception + { + WritableByteChannel wbc = new WritableByteChannel() + { + private boolean isOpen = true; + public int write(ByteBuffer src) throws IOException + { + int size = src.limit(); + serializedFile.writeBytes(src); + return size; + } + + public boolean isOpen() + { + return isOpen; + } + + public void close() throws IOException + { + isOpen = false; + } + }; + + return new EmbeddedChannel(new ChannelOutboundHandlerAdapter() { + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception + { + ((NonClosingDefaultFileRegion) msg).transferTo(wbc, 0); + super.write(ctx, msg, promise); + } + }); + } + + private static List getStreamableComponents(SSTableReader sstable) + { + List result = new ArrayList<>(STREAM_COMPONENTS.size()); + for (Component component : STREAM_COMPONENTS) + { + File file = new File(sstable.descriptor.filenameFor(component)); + if (file.exists()) + { + result.add(new ComponentInfo(component.type, file.length())); + } + } + + return result; + } + + private StreamSession setupStreamingSessionForTest() + { + StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new DefaultConnectionFactory(), false, null, PreviewKind.NONE); + StreamResultFuture future = StreamResultFuture.init(UUID.randomUUID(), StreamOperation.BOOTSTRAP, Collections.emptyList(), streamCoordinator); + + InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); + streamCoordinator.addSessionInfo(new SessionInfo(peer, 0, peer, Collections.emptyList(), Collections.emptyList(), StreamSession.State.INITIALIZED)); + + StreamSession session = streamCoordinator.getOrCreateNextSession(peer); + session.init(future); + return session; + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java new file mode 100644 index 000000000000..53a65b30fcc0 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -0,0 +1,101 @@ +/* + * 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.streaming; + +import java.util.Arrays; +import java.util.List; + +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.RowUpdateBuilder; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.schema.CachingParams; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.streaming.StreamOperation; +import org.apache.cassandra.utils.ByteBufferUtil; + +import static org.junit.Assert.assertTrue; + +public class CassandraOutgoingFileTest +{ + public static final String KEYSPACE = "CassandraOutgoingFileTest"; + public static final String CF_STANDARD = "Standard1"; + public static final String CF_INDEXED = "Indexed1"; + public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval"; + + private static SSTableReader sstable; + + @BeforeClass + public static void defineSchemaAndPrepareSSTable() + { + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE, + KeyspaceParams.simple(1), + SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARD), + SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEXED, true), + SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDLOWINDEXINTERVAL) + .minIndexInterval(8) + .maxIndexInterval(256) + .caching(CachingParams.CACHE_NOTHING)); + + Keyspace keyspace = Keyspace.open(KEYSPACE); + ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1"); + + // insert data and compact to a single sstable + CompactionManager.instance.disableAutoCompaction(); + for (int j = 0; j < 10; j++) + { + new RowUpdateBuilder(store.metadata(), j, String.valueOf(j)) + .clustering("0") + .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) + .build() + .applyUnsafe(); + } + store.forceBlockingFlush(); + CompactionManager.instance.performMaximal(store, false); + + sstable = store.getLiveSSTables().iterator().next(); + } + + @Test + public void testCompleteRangeTriggersFullStreaming() + { + Token minToken = sstable.first.getToken(); + Token maxToken = sstable.last.getToken(); + + Range requestedRange = new Range<>(minToken, maxToken); + + Range.normalize(Arrays.asList(requestedRange)); + + List sections = sstable.getPositionsForRanges(Arrays.asList(requestedRange)); + + CassandraOutgoingFile cof = new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.ref(), sections, + Arrays.asList(requestedRange), sstable.estimatedKeys()); + + assertTrue(cof.shouldStreamFullSSTable()); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index 289bb0f74b2d..ffbf78bd1653 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -19,12 +19,15 @@ package org.apache.cassandra.db.streaming; import java.util.ArrayList; +import java.util.List; +import com.google.common.collect.ImmutableList; import org.junit.Test; import org.apache.cassandra.cql3.statements.CreateTableStatement; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.io.compress.CompressionMetadata; +import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.io.sstable.format.big.BigFormat; import org.apache.cassandra.schema.TableMetadata; @@ -47,4 +50,24 @@ public void serializerTest() SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); } + + @Test + public void serializerTest_FullSSTableTransfer() + { + String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; + TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); + + List ci = ImmutableList.of(new ComponentInfo(Component.Type.DATA, 100)); + + CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, + SSTableFormat.Type.BIG, + 0, + new ArrayList<>(), + ((CompressionMetadata) null), + 0, + SerializationHeader.makeWithoutStats(metadata).toComponent(), + ci, true); + + SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); + } } diff --git a/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java b/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java new file mode 100644 index 000000000000..79f2424fed58 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java @@ -0,0 +1,62 @@ +/* + * 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.streaming; + +import java.io.IOException; + +import org.junit.Test; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.async.ByteBufDataInputPlus; +import org.apache.cassandra.net.async.ByteBufDataOutputPlus; +import org.apache.cassandra.serializers.SerializationUtils; + +import static org.junit.Assert.assertNotEquals; + +public class ComponentInfoTest +{ + @Test + public void testSerialization() + { + ComponentInfo expected = new ComponentInfo(Component.Type.DATA, 100); + SerializationUtils.assertSerializationCycle(expected, ComponentInfo.serializer); + } + + @Test(expected = AssertionError.class) + public void testSerialization_FailsOnBadBytes() throws IOException + { + ByteBuf buf = Unpooled.buffer(512); + ComponentInfo expected = new ComponentInfo(Component.Type.DATA, 100); + + DataOutputPlus output = new ByteBufDataOutputPlus(buf); + ComponentInfo.serializer.serialize(expected, output, MessagingService.VERSION_40); + + buf.setInt(0, -100); + + DataInputPlus input = new ByteBufDataInputPlus(buf); + ComponentInfo actual = ComponentInfo.serializer.deserialize(input, MessagingService.VERSION_40); + + assertNotEquals(expected, actual); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java index cee880216c8e..fccb34439e71 100644 --- a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java @@ -69,7 +69,10 @@ private TestableBTW() private TestableBTW(Descriptor desc) { - this(desc, SSTableTxnWriter.create(cfs, desc, 0, 0, null, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS))); + this(desc, SSTableTxnWriter.create(cfs, desc, 0, 0, null, + new SerializationHeader(true, cfs.metadata(), + cfs.metadata().regularAndStaticColumns(), + EncodingStats.NO_STATS))); } private TestableBTW(Descriptor desc, SSTableTxnWriter sw) diff --git a/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java b/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java new file mode 100644 index 000000000000..a6a7c3b666ba --- /dev/null +++ b/test/unit/org/apache/cassandra/io/sstable/ComponentTest.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.io.sstable; + +import org.junit.Test; + +import static org.junit.Assert.*; + +public class ComponentTest +{ + @Test + public void testTypeCreationFromRepresentation_ValidId() + { + for (int i = 1; i < 10; i++) + assertTrue(Component.Type.fromRepresentation((byte) i) != Component.Type.CUSTOM); + } + + @Test + public void testTypeCreationFromRepresentation_InvalidIds() + { + assertTrue(Component.Type.fromRepresentation((byte) -1) == Component.Type.CUSTOM); + assertTrue(Component.Type.fromRepresentation((byte) 11) == Component.Type.CUSTOM); + assertTrue(Component.Type.fromRepresentation((byte) 12) == Component.Type.CUSTOM); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java index fcc9191c1f30..0e801ee6b834 100644 --- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java @@ -22,6 +22,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Random; import java.util.UUID; @@ -318,8 +319,9 @@ private void streamLegacyTable(String tablePattern, String legacyVersion) throws List streams = Lists.newArrayList(new CassandraOutgoingFile(StreamOperation.OTHER, sstable.ref(), sstable.getPositionsForRanges(ranges), + ranges, sstable.estimatedKeysForRanges(ranges))); - new StreamPlan(StreamOperation.OTHER).transferStreams(FBUtilities.getBroadcastAddressAndPort(), streams).execute().get(); + new StreamPlan(StreamOperation.OTHER, false).transferStreams(FBUtilities.getBroadcastAddressAndPort(), streams).execute().get(); } private static void truncateLegacyTables(String legacyVersion) throws Exception diff --git a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java new file mode 100644 index 000000000000..6481963d6d73 --- /dev/null +++ b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java @@ -0,0 +1,196 @@ +/* + * 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.ByteArrayInputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.File; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +import javax.xml.crypto.Data; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.UpdateBuilder; +import org.apache.cassandra.Util; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.RowUpdateBuilder; +import org.apache.cassandra.db.SerializationHeader; +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.marshal.AsciiType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.rows.EncodingStats; +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.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.schema.CachingParams; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadataRef; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Pair; + +import static org.apache.cassandra.io.util.DataInputPlus.*; +import static org.junit.Assert.*; + +public class BigTableBlockWriterTest +{ + public static final String KEYSPACE1 = "BigTableBlockWriterTest"; + public static final String CF_STANDARD = "Standard1"; + public static final String CF_STANDARD2 = "Standard2"; + public static final String CF_INDEXED = "Indexed1"; + public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval"; + + private static ColumnFamilyStore cfs; + + @BeforeClass + public static void defineSchema() throws Exception + { + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE1, + KeyspaceParams.simple(1), + SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD), + SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2), + SchemaLoader.compositeIndexCFMD(KEYSPACE1, CF_INDEXED, true), + SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLOWINDEXINTERVAL) + .minIndexInterval(8) + .maxIndexInterval(256) + .caching(CachingParams.CACHE_NOTHING)); + } + + @Test + public void writeDataFile() + { + String ks = KEYSPACE1; + String cf = "Standard1"; + + // clear and create just one sstable for this test + Keyspace keyspace = Keyspace.open(ks); + ColumnFamilyStore store = keyspace.getColumnFamilyStore(cf); + store.clearUnsafe(); + store.disableAutoCompaction(); + + DecoratedKey firstKey = null, lastKey = null; + long timestamp = System.currentTimeMillis(); + for (int i = 0; i < store.metadata().params.minIndexInterval; i++) + { + DecoratedKey key = Util.dk(String.valueOf(i)); + if (firstKey == null) + firstKey = key; + if (lastKey == null) + lastKey = key; + if (store.metadata().partitionKeyType.compare(lastKey.getKey(), key.getKey()) < 0) + lastKey = key; + + new RowUpdateBuilder(store.metadata(), timestamp, key.getKey()) + .clustering("col") + .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) + .build() + .applyUnsafe(); + } + store.forceBlockingFlush(); + + SSTableReader sstable = store.getLiveSSTables().iterator().next(); + File dir = store.getDirectories().getDirectoryForNewSSTables(); + Descriptor desc = store.newSSTableDescriptor(dir); + TableMetadataRef metadata = Schema.instance.getTableMetadataRef(desc); + + LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.STREAM); + Set componentsToWrite = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, + Component.STATS); + + BigTableBlockWriter btbw = new BigTableBlockWriter(desc, metadata, txn, componentsToWrite); + + for (Component component : componentsToWrite) + { + if (Files.exists(Paths.get(desc.filenameFor(component)))) + { + Pair pair = getSSTableComponentData(sstable, component); + btbw.writeComponent(component.type, pair.left, pair.right); + } + } + + Collection readers = btbw.finish(true); + + for (SSTableReader reader : readers) + { + System.out.printf("File: %s Generation: %s\n", reader.descriptor.filenameFor(Component.DATA), + reader.descriptor.generation); + } + + SSTableReader reader = readers.toArray(new SSTableReader[0])[0]; + + assertNotEquals(sstable.getFilename(), reader.getFilename()); + assertEquals(sstable.estimatedKeys(), reader.estimatedKeys()); + assertEquals(sstable.isPendingRepair(), reader.isPendingRepair()); + } + + private Pair getSSTableComponentData(SSTableReader sstable, Component component) + { + FileHandle componentFile = new FileHandle.Builder(sstable.descriptor.filenameFor(component)) + .bufferSize(1024).complete(); + + ByteBuffer buffer = ByteBuffer.allocate((int) componentFile.channel.size()); + componentFile.channel.read(buffer, 0); + DataInputStreamPlus is = new DataInputStreamPlus(new ByteArrayInputStream(buffer.array())); + + return Pair.create(is, componentFile.channel.size()); + } + + public static ByteBuffer random(int i, int size) + { + byte[] bytes = new byte[size + 4]; + ThreadLocalRandom.current().nextBytes(bytes); + ByteBuffer r = ByteBuffer.wrap(bytes); + r.putInt(0, i); + return r; + } + + public static void truncate(ColumnFamilyStore cfs) + { + cfs.truncateBlocking(); + LifecycleTransaction.waitForDeletions(); + Uninterruptibles.sleepUninterruptibly(10L, TimeUnit.MILLISECONDS); + assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount()); + assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount()); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/serializers/SerializationUtils.java b/test/unit/org/apache/cassandra/serializers/SerializationUtils.java index 7ce4ec5d41bd..b88b56f003de 100644 --- a/test/unit/org/apache/cassandra/serializers/SerializationUtils.java +++ b/test/unit/org/apache/cassandra/serializers/SerializationUtils.java @@ -63,6 +63,5 @@ public static void assertSerializationCycle(T src, IVersionedSerializer s public static void assertSerializationCycle(T src, IVersionedSerializer serializer) { assertSerializationCycle(src, serializer, MessagingService.current_version); - } } diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java index 78b30949a52d..8ebe333622b8 100644 --- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java +++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java @@ -92,7 +92,7 @@ public void testScheduleTimeout() throws Exception { List> ranges = new ArrayList<>(); ranges.add(new Range<>(sstable.first.getToken(), sstable.last.getToken())); - task.addTransferStream(new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.selfRef(), sstable.getPositionsForRanges(ranges), 1)); + task.addTransferStream(new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.selfRef(), sstable.getPositionsForRanges(ranges), ranges, 1)); } assertEquals(2, task.getTotalNumberOfFiles()); @@ -144,7 +144,7 @@ public void testFailSessionDuringTransferShouldNotReleaseReferences() throws Exc ranges.add(new Range<>(sstable.first.getToken(), sstable.last.getToken())); Ref ref = sstable.selfRef(); refs.add(ref); - task.addTransferStream(new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, ref, sstable.getPositionsForRanges(ranges), 1)); + task.addTransferStream(new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, ref, sstable.getPositionsForRanges(ranges), ranges, 1)); } assertEquals(2, task.getTotalNumberOfFiles()); diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java index 72b9cbe7b3b8..84e9eaffff26 100644 --- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java +++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java @@ -114,7 +114,7 @@ public static void defineSchema() throws Exception @Test public void testEmptyStreamPlan() throws Exception { - StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER).execute(); + StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER, false).execute(); final UUID planId = futureResult.planId; Futures.addCallback(futureResult, new FutureCallback() { @@ -143,7 +143,7 @@ public void testRequestEmpty() throws Exception ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("key1")))); ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken())); - StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER) + StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER, false) .requestRanges(LOCAL, KEYSPACE2, ranges) .execute(); @@ -238,7 +238,7 @@ private void transferRanges(ColumnFamilyStore cfs) throws Exception List> ranges = new ArrayList<>(); // wrapped range ranges.add(new Range(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key0")))); - StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getTableName()); + StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER, false).transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getTableName()); streamPlan.execute().get(); //cannot add ranges after stream session is finished @@ -255,7 +255,7 @@ private void transferRanges(ColumnFamilyStore cfs) throws Exception private void transfer(SSTableReader sstable, List> ranges) throws Exception { - StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferStreams(LOCAL, makeOutgoingStreams(ranges, Refs.tryRef(Arrays.asList(sstable)))); + StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER, false).transferStreams(LOCAL, makeOutgoingStreams(ranges, Refs.tryRef(Arrays.asList(sstable)))); streamPlan.execute().get(); //cannot add files after stream session is finished @@ -278,6 +278,7 @@ private Collection makeOutgoingStreams(StreamOperation operation streams.add(new CassandraOutgoingFile(operation, sstables.get(sstable), sstable.getPositionsForRanges(ranges), + ranges, sstable.estimatedKeysForRanges(ranges))); } return streams; From 17a4f33b47ed545a1968b1ad0b3b6b3727f78649 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 3 Jul 2018 12:10:28 -0700 Subject: [PATCH 02/64] Add missing file --- .../cassandra/io/util/CheckedFunction.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 src/java/org/apache/cassandra/io/util/CheckedFunction.java diff --git a/src/java/org/apache/cassandra/io/util/CheckedFunction.java b/src/java/org/apache/cassandra/io/util/CheckedFunction.java new file mode 100644 index 000000000000..f62ee1eabc29 --- /dev/null +++ b/src/java/org/apache/cassandra/io/util/CheckedFunction.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.util; + +@FunctionalInterface +public interface CheckedFunction { + R apply(T t) throws E; +} From 0069ca22221c449f7216680d8da88a946eec2ffa Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 3 Jul 2018 15:46:32 -0700 Subject: [PATCH 03/64] Clean up --- .../db/streaming/CassandraBlockStreamWriter.java | 5 +++-- .../db/streaming/CassandraOutgoingFile.java | 15 +++++---------- .../org/apache/cassandra/dht/RangeStreamer.java | 1 - .../cassandra/io/util/RebufferingInputStream.java | 8 +------- .../net/async/ByteBufDataOutputStreamPlus.java | 7 ++++--- .../async/RebufferingByteBufDataInputPlus.java | 3 --- .../apache/cassandra/service/StorageService.java | 2 +- test/conf/cassandra.yaml | 1 + .../db/streaming/CassandraOutgoingFileTest.java | 3 ++- .../cassandra/io/sstable/LegacySSTableTest.java | 2 +- .../streaming/StreamingTransferTest.java | 8 ++++---- 11 files changed, 22 insertions(+), 33 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index 59d64954de03..43aa6358b84b 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -47,13 +47,14 @@ public class CassandraBlockStreamWriter implements IStreamWriter protected final SSTableReader sstable; protected final List components; protected final StreamSession session; - + private final StreamRateLimiter limiter; public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, List components) { this.session = session; this.sstable = sstable; this.components = components; + this.limiter = StreamManager.getRateLimiter(session.peer); } /** @@ -85,7 +86,7 @@ public void write(DataOutputStreamPlus output) throws IOException logger.debug("[Stream #{}] Block streaming {}.{} gen {} component {} size {}", session.planId(), sstable.getKeyspaceName(), sstable.getColumnFamilyName(), sstable.descriptor.generation, info.type, length); - bytesRead += byteBufDataOutputStreamPlus.writeToChannel(in); + bytesRead += byteBufDataOutputStreamPlus.writeToChannel(in, limiter); progress += bytesRead; session.progress(sstable.descriptor.filenameFor(Component.parse(info.type.repr)), ProgressInfo.Direction.OUT, bytesRead, diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index f3f89c04d9fb..cb2b96ba09f7 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -41,7 +41,6 @@ import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataOutputStreamPlus; import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.OutgoingStream; import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.streaming.StreamSession; @@ -52,6 +51,11 @@ */ public class CassandraOutgoingFile implements OutgoingStream { + private static final boolean isFullSSTableTransfersEnabled = DatabaseDescriptor.isFullSSTableTransfersEnabled(); + public 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 final Ref ref; private final long estimatedKeys; private final List sections; @@ -61,12 +65,7 @@ public class CassandraOutgoingFile implements OutgoingStream private final List components; private final boolean isFullyContained; - public 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 final List> ranges; - private static final Logger logger = LoggerFactory.getLogger(CassandraOutgoingFile.class); - private static final boolean isFullSSTableTransfersEnabled = DatabaseDescriptor.isFullSSTableTransfersEnabled(); public CassandraOutgoingFile(StreamOperation operation, Ref ref, List sections, Collection> ranges, @@ -183,10 +182,6 @@ private boolean fullyContainedIn(List> requestedRanges, SSTableRead return false; try (KeyIterator iter = new KeyIterator(sstable.descriptor, sstable.metadata())) { - List> ownedRanges = Range.normalize(StorageService.instance.getLocalAndPendingRanges(sstable.getKeyspaceName())); - if (ownedRanges.isEmpty()) - return false; - while (iter.hasNext()) { DecoratedKey key = iter.next(); diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java index 2fc6e21db566..110fed639c02 100644 --- a/src/java/org/apache/cassandra/dht/RangeStreamer.java +++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java @@ -431,7 +431,6 @@ public StreamResultFuture fetchAsync() // filter out already streamed ranges Set> availableRanges = stateStore.getAvailableRanges(keyspace, StorageService.instance.getTokenMetadata().partitioner); - if (ranges.removeAll(availableRanges)) { logger.info("Some ranges of {} are already available. Skipping streaming those ranges.", availableRanges); diff --git a/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java b/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java index f0a47fa9bbaa..f8ae4f64d871 100644 --- a/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java +++ b/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java @@ -87,13 +87,7 @@ public int read(byte[] b, int off, int len) throws IOException int remaining = buffer.limit() - position; if (remaining == 0) { - try - { - reBuffer(); - } catch (EOFException e) - { - throw new EOFException("EOF after " + copied + " bytes out of " + len); - } + reBuffer(); position = buffer.position(); remaining = buffer.limit() - position; if (remaining == 0) diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java index 959655adac9d..f2ade13f7fe3 100644 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java @@ -18,7 +18,6 @@ package org.apache.cassandra.net.async; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; @@ -35,11 +34,11 @@ import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; -import io.netty.channel.DefaultFileRegion; import io.netty.util.concurrent.Future; import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; import org.apache.cassandra.io.util.DataOutputStreamPlus; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter; import org.apache.cassandra.streaming.StreamSession; /** @@ -149,7 +148,7 @@ public ChannelFuture writeToChannel(ByteBuffer buffer) throws IOException * @return number of bytes transferred * @throws IOException */ - public long writeToChannel(FileChannel f) throws IOException + public long writeToChannel(FileChannel f, StreamRateLimiter limiter) throws IOException { final long length = f.size(); long bytesTransferred = 0; @@ -162,6 +161,8 @@ public long writeToChannel(FileChannel f) throws IOException if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, toRead, 5, TimeUnit.MINUTES)) throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", toRead)); + limiter.acquire(toRead); + bytesTransferred += toRead; final boolean shouldClose = (bytesTransferred == length); // this is the last buffer, can safely close channel diff --git a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java index cd0f6fdc7e2c..aa4fa19bde68 100644 --- a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java +++ b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java @@ -293,9 +293,6 @@ public long consumeUntil(BufferedDataOutputStreamPlus writer, long len) throws I int result = writer.applyToChannel(c -> c.write(dup)); - if (result == -1) - throw new IOException(String.format("Failed to write to Channel {}", writer)); - buffer.position(position + result); copied += result; } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 1872153ec466..2d3d7340e1d6 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -2816,7 +2816,7 @@ private void restoreReplicaCount(InetAddressAndPort endpoint, final InetAddressA } } - StreamPlan stream = new StreamPlan(StreamOperation.RESTORE_REPLICA_COUNT, false); + StreamPlan stream = new StreamPlan(StreamOperation.RESTORE_REPLICA_COUNT); for (String keyspaceName : rangesToFetch.keySet()) { for (Map.Entry>> entry : rangesToFetch.get(keyspaceName)) diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml index 5893babf36cb..b072730877ce 100644 --- a/test/conf/cassandra.yaml +++ b/test/conf/cassandra.yaml @@ -46,3 +46,4 @@ enable_user_defined_functions: true enable_scripted_user_defined_functions: true prepared_statements_cache_size_mb: 1 corrupted_tombstone_strategy: exception +streaming_full_sstable_transfers_enabled: true \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java index 53a65b30fcc0..44a885135707 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -25,6 +25,7 @@ import org.junit.Test; import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.RowUpdateBuilder; @@ -84,7 +85,7 @@ public static void defineSchemaAndPrepareSSTable() @Test public void testCompleteRangeTriggersFullStreaming() { - Token minToken = sstable.first.getToken(); + Token minToken = sstable.first.getToken().minValue(); Token maxToken = sstable.last.getToken(); Range requestedRange = new Range<>(minToken, maxToken); diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java index 0e801ee6b834..c61ee1fe68ec 100644 --- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java @@ -321,7 +321,7 @@ private void streamLegacyTable(String tablePattern, String legacyVersion) throws sstable.getPositionsForRanges(ranges), ranges, sstable.estimatedKeysForRanges(ranges))); - new StreamPlan(StreamOperation.OTHER, false).transferStreams(FBUtilities.getBroadcastAddressAndPort(), streams).execute().get(); + new StreamPlan(StreamOperation.OTHER).transferStreams(FBUtilities.getBroadcastAddressAndPort(), streams).execute().get(); } private static void truncateLegacyTables(String legacyVersion) throws Exception diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java index 84e9eaffff26..bc501be6b31c 100644 --- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java +++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java @@ -114,7 +114,7 @@ public static void defineSchema() throws Exception @Test public void testEmptyStreamPlan() throws Exception { - StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER, false).execute(); + StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER).execute(); final UUID planId = futureResult.planId; Futures.addCallback(futureResult, new FutureCallback() { @@ -143,7 +143,7 @@ public void testRequestEmpty() throws Exception ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("key1")))); ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken())); - StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER, false) + StreamResultFuture futureResult = new StreamPlan(StreamOperation.OTHER) .requestRanges(LOCAL, KEYSPACE2, ranges) .execute(); @@ -238,7 +238,7 @@ private void transferRanges(ColumnFamilyStore cfs) throws Exception List> ranges = new ArrayList<>(); // wrapped range ranges.add(new Range(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key0")))); - StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER, false).transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getTableName()); + StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferRanges(LOCAL, cfs.keyspace.getName(), ranges, cfs.getTableName()); streamPlan.execute().get(); //cannot add ranges after stream session is finished @@ -255,7 +255,7 @@ private void transferRanges(ColumnFamilyStore cfs) throws Exception private void transfer(SSTableReader sstable, List> ranges) throws Exception { - StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER, false).transferStreams(LOCAL, makeOutgoingStreams(ranges, Refs.tryRef(Arrays.asList(sstable)))); + StreamPlan streamPlan = new StreamPlan(StreamOperation.OTHER).transferStreams(LOCAL, makeOutgoingStreams(ranges, Refs.tryRef(Arrays.asList(sstable)))); streamPlan.execute().get(); //cannot add files after stream session is finished From d87a494509e9896c556f07a39ee0a3747adf9d79 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 3 Jul 2018 16:10:23 -0700 Subject: [PATCH 04/64] Fix resource leak warning --- .../streaming/CassandraBlockStreamWriter.java | 1 + .../async/ByteBufDataOutputStreamPlus.java | 41 +++++++++++-------- 2 files changed, 26 insertions(+), 16 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index 43aa6358b84b..1b999095b6fa 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -76,6 +76,7 @@ public void write(DataOutputStreamPlus output) throws IOException for (ComponentInfo info : components) { + @SuppressWarnings("resource") // this is closed after the file is transferred by ByteBufDataOutputStreamPlus FileChannel in = new RandomAccessFile(sstable.descriptor.filenameFor(Component.parse(info.type.repr)), "r").getChannel(); // Total Length to transmit for this file diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java index f2ade13f7fe3..777bc3e73522 100644 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java @@ -153,29 +153,38 @@ public long writeToChannel(FileChannel f, StreamRateLimiter limiter) throws IOEx final long length = f.size(); long bytesTransferred = 0; - while (bytesTransferred < length) + try { - int toRead = (int) Math.min(bufferSize, length - bytesTransferred); - NonClosingDefaultFileRegion fileRegion = new NonClosingDefaultFileRegion(f, bytesTransferred, toRead); + while (bytesTransferred < length) + { + int toRead = (int) Math.min(bufferSize, length - bytesTransferred); + NonClosingDefaultFileRegion fileRegion = new NonClosingDefaultFileRegion(f, bytesTransferred, toRead); - if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, toRead, 5, TimeUnit.MINUTES)) - throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", toRead)); + if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, toRead, 5, TimeUnit.MINUTES)) + throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", toRead)); - limiter.acquire(toRead); + limiter.acquire(toRead); - bytesTransferred += toRead; - final boolean shouldClose = (bytesTransferred == length); // this is the last buffer, can safely close channel + bytesTransferred += toRead; + final boolean shouldClose = (bytesTransferred == length); // this is the last buffer, can safely close channel - channel.writeAndFlush(fileRegion).addListener(future -> { - handleBuffer(future, toRead); + channel.writeAndFlush(fileRegion).addListener(future -> { + handleBuffer(future, toRead); - if ((shouldClose || !future.isSuccess()) && f.isOpen()) - f.close(); - }); - logger.trace("{} of {} (toRead {} cs {})", bytesTransferred, length, toRead, f.isOpen()); - } + if ((shouldClose || !future.isSuccess()) && f.isOpen()) + f.close(); + }); + logger.trace("{} of {} (toRead {} cs {})", bytesTransferred, length, toRead, f.isOpen()); + } - return bytesTransferred; + return bytesTransferred; + } catch (Exception e) + { + if (f.isOpen()) + f.close(); + + throw e; + } } @Override From b28797fa5e9dbcecff388a0e5c4ab633fea7aa7e Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 3 Jul 2018 17:07:38 -0700 Subject: [PATCH 05/64] Fix unit tests --- src/java/org/apache/cassandra/io/util/SequentialWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java index c773d9b1974c..3eb1a7d81500 100644 --- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java +++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java @@ -138,7 +138,7 @@ public SequentialWriter(File file) */ public SequentialWriter(File file, SequentialWriterOption option) { - this(file, option, false); + this(file, option, true); } /** From 2186d435f56f101eb6ce615408ef25c372abae4c Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 3 Jul 2018 16:42:15 -0700 Subject: [PATCH 06/64] Circle CI --- .circleci/config.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 5a84f724fcf8..410a9be1f1c7 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -58,11 +58,11 @@ with_dtest_jobs_only: &with_dtest_jobs_only - build # Set env_settings, env_vars, and workflows/build_and_run_tests based on environment env_settings: &env_settings - <<: *default_env_settings - #<<: *high_capacity_env_settings + #<<: *default_env_settings + <<: *high_capacity_env_settings env_vars: &env_vars - <<: *resource_constrained_env_vars - #<<: *high_capacity_env_vars + #<<: *resource_constrained_env_vars + <<: *high_capacity_env_vars workflows: version: 2 build_and_run_tests: *default_jobs From 071c436a62bcd75f3f0cb746c4cfb82383ae7644 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 3 Jul 2018 17:42:32 -0700 Subject: [PATCH 07/64] Enable dtests for CircleCI --- .circleci/config.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 410a9be1f1c7..76a2c9f84178 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -65,9 +65,9 @@ env_vars: &env_vars <<: *high_capacity_env_vars workflows: version: 2 - build_and_run_tests: *default_jobs + #build_and_run_tests: *default_jobs #build_and_run_tests: *with_dtest_jobs_only - #build_and_run_tests: *with_dtest_jobs + build_and_run_tests: *with_dtest_jobs docker_image: &docker_image kjellman/cassandra-test:0.4.3 version: 2 jobs: From 2a0dfd35f87c3fb410973b4e2c5c8021141100c7 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 4 Jul 2018 02:10:17 -0700 Subject: [PATCH 08/64] Use a synchronized map instead of concurrent map to preserve order --- src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java index 0162f5ab0772..22467dddb91c 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java @@ -20,6 +20,7 @@ import java.io.File; import java.util.Collection; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -46,7 +47,7 @@ public class LogReplicaSet implements AutoCloseable { private static final Logger logger = LoggerFactory.getLogger(LogReplicaSet.class); - private final Map replicasByFile = new ConcurrentHashMap<>(); // TODO: Hack until we fix CASSANDRA-14554 + private final Map replicasByFile = Collections.synchronizedMap(new LinkedHashMap<>()); // TODO: Hack until we fix CASSANDRA-14554 private Collection replicas() { From bfa25ca07eb200e687eec685aa61b06e364dfdbf Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 10 Jul 2018 12:10:55 -0700 Subject: [PATCH 09/64] Code review comments --- src/java/org/apache/cassandra/io/sstable/Component.java | 3 ++- src/java/org/apache/cassandra/io/util/CheckedFunction.java | 3 ++- .../apache/cassandra/io/util/RebufferingInputStream.java | 6 ++---- .../net/async/RebufferingByteBufDataInputPlus.java | 3 ++- src/java/org/apache/cassandra/service/StorageService.java | 4 ++-- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/java/org/apache/cassandra/io/sstable/Component.java b/src/java/org/apache/cassandra/io/sstable/Component.java index 40bcaf6a7dfc..a27fda1cd4c5 100644 --- a/src/java/org/apache/cassandra/io/sstable/Component.java +++ b/src/java/org/apache/cassandra/io/sstable/Component.java @@ -87,7 +87,8 @@ public static Type fromRepresentation(byte id) return idToType.getOrDefault(id, CUSTOM); } - static { + static + { Type[] values = Type.values(); Map result = new HashMap<>(values.length); for (Type t : values) diff --git a/src/java/org/apache/cassandra/io/util/CheckedFunction.java b/src/java/org/apache/cassandra/io/util/CheckedFunction.java index f62ee1eabc29..ec1ce9fc859b 100644 --- a/src/java/org/apache/cassandra/io/util/CheckedFunction.java +++ b/src/java/org/apache/cassandra/io/util/CheckedFunction.java @@ -19,6 +19,7 @@ package org.apache.cassandra.io.util; @FunctionalInterface -public interface CheckedFunction { +public interface CheckedFunction +{ R apply(T t) throws E; } diff --git a/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java b/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java index f8ae4f64d871..ef5188838f7c 100644 --- a/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java +++ b/src/java/org/apache/cassandra/io/util/RebufferingInputStream.java @@ -25,14 +25,12 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; +import com.google.common.base.Preconditions; + import net.nicoulaj.compilecommand.annotations.DontInline; -import org.apache.cassandra.io.FSError; import org.apache.cassandra.utils.FastByteOperations; -import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.vint.VIntCoding; -import com.google.common.base.Preconditions; - /** * Rough equivalent of BufferedInputStream and DataInputStream wrapping a ByteBuffer that can be refilled * via rebuffer. Implementations provide this buffer from various channels (socket, file, memory, etc). diff --git a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java index aa4fa19bde68..348a99172f35 100644 --- a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java +++ b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java @@ -274,7 +274,8 @@ public long consumeUntil(BufferedDataOutputStreamPlus writer, long len) throws I try { reBuffer(); - } catch (EOFException e) + } + catch (EOFException e) { throw new EOFException("EOF after " + copied + " bytes out of " + len); } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 2d3d7340e1d6..96fd63f08ba2 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -4236,7 +4236,7 @@ private void move(Token newToken) throws IOException private class RangeRelocator { - private final StreamPlan streamPlan = new StreamPlan(StreamOperation.RELOCATION, false); + private final StreamPlan streamPlan = new StreamPlan(StreamOperation.RELOCATION); private RangeRelocator(Collection tokens, List keyspaceNames) { @@ -5116,7 +5116,7 @@ private Future streamRanges(Map, Inet sessionsToStreamByKeyspace.put(keyspace, rangesPerEndpoint); } - StreamPlan streamPlan = new StreamPlan(StreamOperation.DECOMMISSION, false); + StreamPlan streamPlan = new StreamPlan(StreamOperation.DECOMMISSION); // Vinculate StreamStateStore to current StreamPlan to update transferred ranges per StreamSession streamPlan.listeners(streamStateStore); From c83a72e2e1614fcd784f2207e1527ede395eecab Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 10 Jul 2018 12:41:23 -0700 Subject: [PATCH 10/64] Simplify consumeUntil --- .../RebufferingByteBufDataInputPlus.java | 28 ++++++------------- 1 file changed, 8 insertions(+), 20 deletions(-) diff --git a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java index 348a99172f35..2f8b22843d35 100644 --- a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java +++ b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java @@ -33,9 +33,6 @@ import io.netty.util.ReferenceCountUtil; import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; import org.apache.cassandra.io.util.RebufferingInputStream; -import org.apache.cassandra.io.util.SequentialWriter; -import org.apache.cassandra.utils.ByteBufferUtil; -import org.apache.cassandra.utils.FastByteOperations; public class RebufferingByteBufDataInputPlus extends RebufferingInputStream implements ReadableByteChannel { @@ -267,9 +264,7 @@ public long consumeUntil(BufferedDataOutputStreamPlus writer, long len) throws I long copied = 0; // number of bytes copied while (copied < len) { - int position = buffer.position(); - int remaining = buffer.remaining(); - if (remaining == 0) + if (buffer.remaining() == 0) { try { @@ -279,23 +274,16 @@ public long consumeUntil(BufferedDataOutputStreamPlus writer, long len) throws I { throw new EOFException("EOF after " + copied + " bytes out of " + len); } - position = buffer.position(); - remaining = buffer.remaining(); - if (remaining == 0) + if (buffer.remaining() == 0) return copied == 0 ? -1 : copied; } - int toCopy = (int) Math.min(len - copied, remaining); - - ByteBuffer dup = buffer.duplicate(); - - if (toCopy < remaining) - dup.limit(dup.position() + toCopy); - - int result = writer.applyToChannel(c -> c.write(dup)); - - buffer.position(position + result); - copied += result; + int originalLimit = buffer.limit(); + int toCopy = (int) Math.min(len - copied, buffer.remaining()); + buffer.limit(buffer.position() + toCopy); + int written = writer.applyToChannel(c -> c.write(buffer)); + buffer.limit(originalLimit); + copied += written; } return copied; From 657ce3670a5291decf2939d72d51abe19333ef15 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 01:14:59 -0700 Subject: [PATCH 11/64] Fix data directory selection --- .../apache/cassandra/db/DiskBoundaries.java | 15 +++++ .../streaming/CassandraBlockStreamReader.java | 27 ++++++-- .../db/streaming/CassandraOutgoingFile.java | 6 +- .../db/streaming/CassandraStreamHeader.java | 66 ++++++++++++++----- .../streaming/LongStreamingTest.java | 6 +- .../ZeroCopyStreamingBenchmark.java | 9 ++- .../CassandraBlockStreamWriterTest.java | 2 +- .../streaming/CassandraStreamHeaderTest.java | 5 +- 8 files changed, 102 insertions(+), 34 deletions(-) diff --git a/src/java/org/apache/cassandra/db/DiskBoundaries.java b/src/java/org/apache/cassandra/db/DiskBoundaries.java index 086bc8498bfb..5b548fc80be9 100644 --- a/src/java/org/apache/cassandra/db/DiskBoundaries.java +++ b/src/java/org/apache/cassandra/db/DiskBoundaries.java @@ -129,4 +129,19 @@ public Directories.DataDirectory getCorrectDiskForSSTable(SSTableReader sstable) { return directories.get(getDiskIndex(sstable)); } + + public Directories.DataDirectory getCorrectDiskForKey(DecoratedKey key) + { + if (positions == null) + return null; + + return directories.get(getDiskIndex(key)); + } + + private int getDiskIndex(DecoratedKey key) + { + int pos = Collections.binarySearch(positions, key); + assert pos < 0; + return -pos - 1; + } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index 82ac4780026c..03aefedf987d 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -28,6 +28,7 @@ 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.SerializationHeader; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; @@ -60,6 +61,7 @@ public class CassandraBlockStreamReader implements IStreamReader private final List components; private final SSTableFormat.Type format; private final Version version; + private final DecoratedKey firstKey; public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHeader streamHeader, StreamSession session) { @@ -79,6 +81,7 @@ public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHea this.format = streamHeader.format; this.fileSeqNum = header.sequenceNumber; this.version = streamHeader.version; + this.firstKey = streamHeader.firstKey; } /** @@ -94,6 +97,7 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException long totalSize = totalSize(); ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); + if (cfs == null) { // schema was dropped during streaming @@ -140,22 +144,33 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException } } - @SuppressWarnings("resource") - protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize, Set componentsToWrite) throws IOException + private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOException { Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize); if (localDir == null) throw new IOException(String.format("Insufficient disk space to store %s", FBUtilities.prettyPrintMemory(totalSize))); - StreamReceiver streamReceiver = session.getAggregator(tableId); + Directories.DataDirectory dataDir = cfs.getDiskBoundaries().getCorrectDiskForKey(firstKey); + File dir = cfs.getDirectories().getDirectoryForNewSSTables(); + + if (dataDir != null) + dir = dataDir.location; + + return dir; + } + + @SuppressWarnings("resource") + protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize, Set componentsToWrite) throws IOException + { + File dataDir = getDataDir(cfs, totalSize); + + StreamReceiver streamReceiver = session.getAggregator(tableId); assert streamReceiver instanceof CassandraStreamReceiver; LifecycleTransaction txn = CassandraStreamReceiver.fromReceiver(session.getAggregator(tableId)).getTransaction(); - // TODO: Is this the correct directory? - File dir = cfs.getDirectories().getDirectoryForNewSSTables(); - Descriptor desc = cfs.newSSTableDescriptor(dir, version, format); + Descriptor desc = cfs.newSSTableDescriptor(dataDir, version, format); logger.debug("[Table #{}] {} Components to write - {}", tableId, desc.filenameFor(Component.DATA), componentsToWrite); BigTableBlockWriter writer = new BigTableBlockWriter(desc, cfs.metadata, txn, componentsToWrite); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index cb2b96ba09f7..8e219fd202eb 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -29,8 +29,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.DecoratedKey; @@ -88,7 +86,9 @@ public CassandraOutgoingFile(StreamOperation operation, Ref ref, sections, sstable.compression ? sstable.getCompressionMetadata() : null, keepSSTableLevel ? sstable.getSSTableLevel() : 0, - sstable.header.toComponent(), components, shouldStreamFullSSTable()); + sstable.header.toComponent(), components, shouldStreamFullSSTable(), + sstable.first, + sstable.metadata().id); } private static List getComponents(SSTableReader sstable) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index 9f4a0c80dc24..7e02051d0132 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -19,10 +19,13 @@ package org.apache.cassandra.db.streaming; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Objects; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.IVersionedSerializer; @@ -32,6 +35,8 @@ import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.utils.ByteBufferUtil; public class CassandraStreamHeader { @@ -54,16 +59,20 @@ public class CassandraStreamHeader /* flag indicating whether this is a partial or full sstable transfer */ public final boolean fullStream; + /* first token of the sstable required for faster streaming */ + public final DecoratedKey firstKey; + public final TableId tableId; + public final List components; /* cached size value */ private transient final long size; public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, - List sections, CompressionMetadata compressionMetadata, - CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header, - List components, - boolean fullStream) + List sections, CompressionMetadata compressionMetadata, + CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header, + List components, + boolean fullStream, DecoratedKey firstKey, TableId tableId) { this.version = version; this.format = format; @@ -75,36 +84,41 @@ public CassandraStreamHeader(Version version, SSTableFormat.Type format, long es this.header = header; this.fullStream = fullStream; this.components = components; - + this.firstKey = firstKey; + this.tableId = tableId; this.size = calculateSize(); } private CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionMetadata compressionMetadata, - CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header) + CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header, TableId tableId) { - this(version, format, estimatedKeys, sections, compressionMetadata, compressionInfo, sstableLevel, header, null, false); + this(version, format, estimatedKeys, sections, compressionMetadata, compressionInfo, sstableLevel, header, null, false, null, tableId); } public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionMetadata compressionMetadata, - int sstableLevel, SerializationHeader.Component header) + int sstableLevel, SerializationHeader.Component header, TableId tableId) { - this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header); + this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header, tableId); } public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionMetadata compressionMetadata, - int sstableLevel, SerializationHeader.Component header, List components, boolean fullStream) + int sstableLevel, SerializationHeader.Component header, List components, + boolean fullStream, DecoratedKey firstKey, TableId tableId) { - this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header, components, fullStream); + this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header, + components, fullStream, firstKey, tableId); } public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionInfo compressionInfo, - int sstableLevel, SerializationHeader.Component header, List components, boolean fullStream) + int sstableLevel, SerializationHeader.Component header, List components, + boolean fullStream, DecoratedKey firstKey, TableId tableId) { - this(version, format, estimatedKeys, sections, null, compressionInfo, sstableLevel, header, components, fullStream); + this(version, format, estimatedKeys, sections, null, compressionInfo, sstableLevel, header, + components, fullStream, firstKey, tableId); } public boolean isCompressed() @@ -183,12 +197,15 @@ public boolean equals(Object o) Objects.equals(sections, that.sections) && Objects.equals(compressionInfo, that.compressionInfo) && Objects.equals(header, that.header) && - Objects.equals(components, that.components); + Objects.equals(components, that.components) && + Objects.equals(firstKey, that.firstKey) && + Objects.equals(tableId, that.tableId); } public int hashCode() { - return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, components, fullStream); + return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, components, + fullStream, firstKey, tableId); } public static final IVersionedSerializer serializer = new IVersionedSerializer() @@ -211,12 +228,16 @@ public void serialize(CassandraStreamHeader header, DataOutputPlus out, int vers SerializationHeader.serializer.serialize(header.version, header.header, out); + header.tableId.serialize(out); out.writeBoolean(header.fullStream); + if (header.fullStream) { out.writeInt(header.components.size()); for (ComponentInfo info : header.components) ComponentInfo.serializer.serialize(info, out, version); + + ByteBufferUtil.writeWithShortLength(header.firstKey.getKey(), out); } } @@ -235,8 +256,10 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws I SerializationHeader.Component header = SerializationHeader.serializer.deserialize(sstableVersion, in); + TableId tableId = TableId.deserialize(in); boolean fullStream = in.readBoolean(); List components = null; + DecoratedKey firstKey = null; if (fullStream) { @@ -245,9 +268,18 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws I for (int i=0; i < ncomp; i++) components.add(ComponentInfo.serializer.deserialize(in, version)); + + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); + + if (cfs == null) + throw new IllegalStateException(String.format("ColumnFamily for tableId {} does not exist", tableId)); + + ByteBuffer keyBuf = ByteBufferUtil.readWithShortLength(in); + firstKey = cfs.getPartitioner().decorateKey(keyBuf); } - return new CassandraStreamHeader(sstableVersion, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, components, fullStream); + return new CassandraStreamHeader(sstableVersion, format, estimatedKeys, sections, compressionInfo, + sstableLevel, header, components, fullStream, firstKey, tableId); } public long serializedSize(CassandraStreamHeader header, int version) @@ -275,6 +307,8 @@ public long serializedSize(CassandraStreamHeader header, int version) size += TypeSizes.sizeof(header.components.size()); for (ComponentInfo info : header.components) size += ComponentInfo.serializer.serializedSize(info, version); + + size += ByteBufferUtil.serializedSizeWithShortLength(header.firstKey.getKey()); } return size; diff --git a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java index bd7ef20893dc..01e67f0d3209 100644 --- a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java +++ b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java @@ -29,10 +29,7 @@ import org.junit.Test; import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.schema.CompressionParams; -import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.schema.Schema; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.Keyspace; @@ -41,6 +38,9 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.io.sstable.CQLSSTableWriter; import org.apache.cassandra.io.sstable.SSTableLoader; +import org.apache.cassandra.schema.CompressionParams; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.OutputHandler; diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index 101862be2975..3cd7e46cbb11 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -50,7 +50,6 @@ import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.ByteBufDataInputPlus; import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; import org.apache.cassandra.net.async.NonClosingDefaultFileRegion; import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; @@ -65,7 +64,6 @@ import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.StreamSummary; -import org.apache.cassandra.streaming.messages.IncomingStreamMessage; import org.apache.cassandra.streaming.messages.StreamMessageHeader; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; @@ -177,7 +175,12 @@ public void stream(BenchmarkState state) throws Exception session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, serializedFile.readableBytes())); - CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), getStreamableComponents(sstable), true); + CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, + sstable.estimatedKeys(), Collections.emptyList(), + (CompressionInfo) null, 0, sstable.header.toComponent(), + getStreamableComponents(sstable), true, sstable.first, + sstable.metadata().id); + CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java index 81c4e0596492..158f26924c7e 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java @@ -146,7 +146,7 @@ public void testBlockReadingAndWritingOverWire() throws Exception session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104)); - CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), getStreamableComponents(sstable), true); + CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), getStreamableComponents(sstable), true, sstable.first, sstable.metadata().id); CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index ffbf78bd1653..fe37bd9fc9e3 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -46,7 +46,8 @@ public void serializerTest() new ArrayList<>(), ((CompressionMetadata) null), 0, - SerializationHeader.makeWithoutStats(metadata).toComponent()); + SerializationHeader.makeWithoutStats(metadata).toComponent(), + metadata.id); SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); } @@ -66,7 +67,7 @@ public void serializerTest_FullSSTableTransfer() ((CompressionMetadata) null), 0, SerializationHeader.makeWithoutStats(metadata).toComponent(), - ci, true); + ci, true, null, metadata.id); SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); } From 7588777fd9ed086f1153fb214bd7f8bdb5f8ea5f Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 01:24:09 -0700 Subject: [PATCH 12/64] update circleci config to point to dtest fork --- .circleci/config.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index 76a2c9f84178..69115ce479de 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -206,7 +206,7 @@ jobs: name: Clone Cassandra dtest Repository (via git) command: | export LANG=en_US.UTF-8 - git clone --single-branch --branch master --depth 1 git://github.com/apache/cassandra-dtest.git ~/cassandra-dtest + git clone --single-branch --branch faster-streaming-rev2 --depth 1 git://github.com/dineshjoshi/cassandra-dtest-1.git ~/cassandra-dtest - run: name: Configure virtualenv and python Dependencies command: | @@ -284,7 +284,7 @@ jobs: name: Clone Cassandra dtest Repository (via git) command: | export LANG=en_US.UTF-8 - git clone --single-branch --branch master --depth 1 git://github.com/apache/cassandra-dtest.git ~/cassandra-dtest + git clone --single-branch --branch faster-streaming-rev2 --depth 1 git://github.com/dineshjoshi/cassandra-dtest-1.git ~/cassandra-dtest - run: name: Configure virtualenv and python Dependencies command: | From 212f8dba23a0403823a2d78246a0f218313f50f9 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 01:54:23 -0700 Subject: [PATCH 13/64] Fix serialized size --- .../org/apache/cassandra/db/streaming/CassandraStreamHeader.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index 7e02051d0132..e5159daf83bf 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -300,6 +300,7 @@ public long serializedSize(CassandraStreamHeader header, int version) size += SerializationHeader.serializer.serializedSize(header.version, header.header); + size += header.tableId.serializedSize(); size += TypeSizes.sizeof(header.fullStream); if (header.fullStream) From d323e9f006cca4b63f7675cc5ebe75070e150f5b Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 02:42:37 -0700 Subject: [PATCH 14/64] Testing --- .../streaming/CassandraBlockStreamReader.java | 4 +- .../streaming/CassandraStreamHeaderTest.java | 40 ++++++++++--------- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index 03aefedf987d..9d65c93cfcc7 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -154,8 +154,8 @@ private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOExceptio File dir = cfs.getDirectories().getDirectoryForNewSSTables(); - if (dataDir != null) - dir = dataDir.location; +// if (dataDir != null) +// dir = dataDir.location; return dir; } diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index fe37bd9fc9e3..4ce344c70637 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -26,12 +26,14 @@ import org.apache.cassandra.cql3.statements.CreateTableStatement; import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.io.sstable.format.big.BigFormat; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.serializers.SerializationUtils; +import org.apache.cassandra.utils.ByteBufferUtil; public class CassandraStreamHeaderTest { @@ -52,23 +54,23 @@ public void serializerTest() SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); } - @Test - public void serializerTest_FullSSTableTransfer() - { - String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; - TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); - - List ci = ImmutableList.of(new ComponentInfo(Component.Type.DATA, 100)); - - CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, - SSTableFormat.Type.BIG, - 0, - new ArrayList<>(), - ((CompressionMetadata) null), - 0, - SerializationHeader.makeWithoutStats(metadata).toComponent(), - ci, true, null, metadata.id); - - SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); - } +// @Test +// public void serializerTest_FullSSTableTransfer() +// { +// String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; +// TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); +// +// List ci = ImmutableList.of(new ComponentInfo(Component.Type.DATA, 100)); +// +// CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, +// SSTableFormat.Type.BIG, +// 0, +// new ArrayList<>(), +// ((CompressionMetadata) null), +// 0, +// SerializationHeader.makeWithoutStats(metadata).toComponent(), +// ci, true, Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER), metadata.id); +// +// SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); +// } } From 9a24738baf28dbd7fd30c5b343c1daf4de00330a Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 03:12:35 -0700 Subject: [PATCH 15/64] Fix directory selection --- .../db/streaming/CassandraBlockStreamReader.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index 9d65c93cfcc7..7a731be9fa0d 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -150,12 +150,10 @@ private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOExceptio if (localDir == null) throw new IOException(String.format("Insufficient disk space to store %s", FBUtilities.prettyPrintMemory(totalSize))); - Directories.DataDirectory dataDir = cfs.getDiskBoundaries().getCorrectDiskForKey(firstKey); + File dir = cfs.getDirectories().getLocationForDisk(cfs.getDiskBoundaries().getCorrectDiskForKey(firstKey)); - File dir = cfs.getDirectories().getDirectoryForNewSSTables(); - -// if (dataDir != null) -// dir = dataDir.location; + if (dir == null) + return cfs.getDirectories().getDirectoryForNewSSTables(); return dir; } From 25ce5def9fd167f2a843e5191fb5530060d206ed Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 13:37:39 -0700 Subject: [PATCH 16/64] Readd test --- .../db/streaming/CassandraStreamHeader.java | 22 +++++--- .../streaming/CassandraStreamHeaderTest.java | 50 ++++++++++++------- 2 files changed, 45 insertions(+), 27 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index e5159daf83bf..deed0fc9252d 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -24,10 +24,13 @@ import java.util.List; import java.util.Objects; -import org.apache.cassandra.db.ColumnFamilyStore; +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.sstable.format.SSTableFormat; @@ -208,7 +211,9 @@ public int hashCode() fullStream, firstKey, tableId); } - public static final IVersionedSerializer serializer = new IVersionedSerializer() + public static final IVersionedSerializer serializer = new CassandraStreamHeaderSerializer(); + + public static class CassandraStreamHeaderSerializer implements IVersionedSerializer { public void serialize(CassandraStreamHeader header, DataOutputPlus out, int version) throws IOException { @@ -242,6 +247,12 @@ public void serialize(CassandraStreamHeader header, DataOutputPlus out, int vers } public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws IOException + { + return deserialize(in, version, DatabaseDescriptor.getPartitioner()); + } + + @VisibleForTesting + public CassandraStreamHeader deserialize(DataInputPlus in, int version, IPartitioner partitioner) throws IOException { Version sstableVersion = SSTableFormat.Type.current().info.getVersion(in.readUTF()); SSTableFormat.Type format = SSTableFormat.Type.validate(in.readUTF()); @@ -269,13 +280,8 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws I for (int i=0; i < ncomp; i++) components.add(ComponentInfo.serializer.deserialize(in, version)); - ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); - - if (cfs == null) - throw new IllegalStateException(String.format("ColumnFamily for tableId {} does not exist", tableId)); - ByteBuffer keyBuf = ByteBufferUtil.readWithShortLength(in); - firstKey = cfs.getPartitioner().decorateKey(keyBuf); + firstKey = partitioner.decorateKey(keyBuf); } return new CassandraStreamHeader(sstableVersion, format, estimatedKeys, sections, compressionInfo, diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index 4ce344c70637..f6cf7e7d19e3 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.streaming; +import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -26,11 +27,13 @@ import org.apache.cassandra.cql3.statements.CreateTableStatement; import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.streaming.CassandraStreamHeader.CassandraStreamHeaderSerializer; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.io.sstable.format.big.BigFormat; +import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.serializers.SerializationUtils; import org.apache.cassandra.utils.ByteBufferUtil; @@ -54,23 +57,32 @@ public void serializerTest() SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); } -// @Test -// public void serializerTest_FullSSTableTransfer() -// { -// String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; -// TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); -// -// List ci = ImmutableList.of(new ComponentInfo(Component.Type.DATA, 100)); -// -// CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, -// SSTableFormat.Type.BIG, -// 0, -// new ArrayList<>(), -// ((CompressionMetadata) null), -// 0, -// SerializationHeader.makeWithoutStats(metadata).toComponent(), -// ci, true, Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER), metadata.id); -// -// SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); -// } + @Test + public void serializerTest_FullSSTableTransfer() + { + String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; + TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); + + List ci = ImmutableList.of(new ComponentInfo(Component.Type.DATA, 100)); + + CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, + SSTableFormat.Type.BIG, + 0, + new ArrayList<>(), + ((CompressionMetadata) null), + 0, + SerializationHeader.makeWithoutStats(metadata).toComponent(), + ci, true, Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER), metadata.id); + + SerializationUtils.assertSerializationCycle(header, new TestableCassandraStreamHeaderSerializer()); + } + + private static class TestableCassandraStreamHeaderSerializer extends CassandraStreamHeaderSerializer + { + @Override + public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws IOException + { + return deserialize(in, version, Murmur3Partitioner.instance); + } + } } From 04d1405c7243bacaf3cc1118a5a6cc272cdbdd14 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 15:12:09 -0700 Subject: [PATCH 17/64] Dont use DatabaseDescriptor to determine Partitioner --- .../db/streaming/CassandraStreamHeader.java | 17 ++++++++++++++--- .../db/streaming/CassandraStreamHeaderTest.java | 2 +- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index deed0fc9252d..aa3474c35362 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -23,10 +23,11 @@ import java.util.ArrayList; import java.util.List; import java.util.Objects; +import java.util.function.Function; import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.TypeSizes; @@ -248,11 +249,17 @@ public void serialize(CassandraStreamHeader header, DataOutputPlus out, int vers public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws IOException { - return deserialize(in, version, DatabaseDescriptor.getPartitioner()); + return deserialize(in, version, tableId -> { + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); + if (cfs != null) + return cfs.getPartitioner(); + + return null; + }); } @VisibleForTesting - public CassandraStreamHeader deserialize(DataInputPlus in, int version, IPartitioner partitioner) throws IOException + public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function partitionerMapper) throws IOException { Version sstableVersion = SSTableFormat.Type.current().info.getVersion(in.readUTF()); SSTableFormat.Type format = SSTableFormat.Type.validate(in.readUTF()); @@ -281,6 +288,10 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version, IPartiti components.add(ComponentInfo.serializer.deserialize(in, version)); ByteBuffer keyBuf = ByteBufferUtil.readWithShortLength(in); + + IPartitioner partitioner = partitionerMapper.apply(tableId); + if (partitioner == null) + throw new IllegalArgumentException(String.format("Could not determine partitioner for tableId {}", tableId)); firstKey = partitioner.decorateKey(keyBuf); } diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index f6cf7e7d19e3..d86f17d3d76a 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -82,7 +82,7 @@ private static class TestableCassandraStreamHeaderSerializer extends CassandraSt @Override public CassandraStreamHeader deserialize(DataInputPlus in, int version) throws IOException { - return deserialize(in, version, Murmur3Partitioner.instance); + return deserialize(in, version, tableId -> Murmur3Partitioner.instance); } } } From 522422efcbc499cea6fdb810473a1bf4a5f72fac Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 16:42:23 -0700 Subject: [PATCH 18/64] Add tests for consumeUntil --- .../RebufferingByteBufDataInputPlusTest.java | 98 +++++++++++++++++++ 1 file changed, 98 insertions(+) diff --git a/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java b/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java index 2961d9aecc4d..69df0403a223 100644 --- a/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java +++ b/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java @@ -21,6 +21,7 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; import org.junit.After; import org.junit.Assert; @@ -28,7 +29,9 @@ import org.junit.Test; import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; public class RebufferingByteBufDataInputPlusTest { @@ -151,4 +154,99 @@ public void available_ClosedButWithBytes() throws EOFException inputPlus.markClose(); Assert.assertEquals(size, inputPlus.available()); } + + @Test + public void consumeUntil_SingleBuffer_Partial_HappyPath() throws IOException + { + consumeUntilTestCycle(1, 8, 0, 4); + } + + @Test + public void consumeUntil_SingleBuffer_AllBytes_HappyPath() throws IOException + { + consumeUntilTestCycle(1, 8, 0, 8); + } + + @Test + public void consumeUntil_MultipleBufferr_Partial_HappyPath() throws IOException + { + consumeUntilTestCycle(2, 8, 0, 13); + } + + @Test + public void consumeUntil_MultipleBuffer_AllBytes_HappyPath() throws IOException + { + consumeUntilTestCycle(2, 8, 0, 16); + } + + @Test(expected = EOFException.class) + public void consumeUntil_SingleBuffer_Fails() throws IOException + { + consumeUntilTestCycle(1, 8, 0, 9); + } + + @Test(expected = EOFException.class) + public void consumeUntil_MultipleBuffer_Fails() throws IOException + { + consumeUntilTestCycle(2, 8, 0, 17); + } + + private void consumeUntilTestCycle(int nBuffs, int buffSize, int startOffset, int len) throws IOException + { + byte[] expectedBytes = new byte[len]; + int count = 0; + for (int j=0; j < nBuffs; j++) + { + ByteBuf buf = channel.alloc().buffer(buffSize); + for (int i = 0; i < buf.capacity(); i++) + { + buf.writeByte(j); + if (count >= startOffset && (count - startOffset) < len) + expectedBytes[count - startOffset] = (byte)j; + count++; + } + + inputPlus.append(buf); + } + inputPlus.append(channel.alloc().buffer(0)); + + TestableWritableByteChannel wbc = new TestableWritableByteChannel(len); + + inputPlus.skipBytesFully(startOffset); + BufferedDataOutputStreamPlus writer = new BufferedDataOutputStreamPlus(wbc); + inputPlus.consumeUntil(writer, len); + + Assert.assertEquals(String.format("Test with {} buffers starting at {} consuming {} bytes", nBuffs, startOffset, + len), len, wbc.writtenBytes.readableBytes()); + + Assert.assertArrayEquals(expectedBytes, wbc.writtenBytes.array()); + } + + private static class TestableWritableByteChannel implements WritableByteChannel + { + private boolean isOpen = true; + public ByteBuf writtenBytes; + + public TestableWritableByteChannel(int initialCapacity) + { + writtenBytes = Unpooled.buffer(initialCapacity); + } + + public int write(ByteBuffer src) throws IOException + { + int size = src.remaining(); + writtenBytes.writeBytes(src); + return size; + } + + public boolean isOpen() + { + return isOpen; + } + + public void close() throws IOException + { + isOpen = false; + } + }; } From a90aad8fb32818744af4894027d7b1d21e156ab6 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 17:54:16 -0700 Subject: [PATCH 19/64] Update BTBW test --- .../format/big/BigTableBlockWriterTest.java | 123 ++++++++++-------- 1 file changed, 69 insertions(+), 54 deletions(-) diff --git a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java index 6481963d6d73..3b5571065688 100644 --- a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java @@ -19,58 +19,51 @@ package org.apache.cassandra.io.sstable.format.big; import java.io.ByteArrayInputStream; -import java.io.DataInput; -import java.io.DataInputStream; import java.io.File; import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; import java.nio.file.Files; -import java.nio.file.Path; import java.nio.file.Paths; import java.util.Collection; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.function.Function; -import javax.xml.crypto.Data; - -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.Uninterruptibles; import org.junit.BeforeClass; import org.junit.Test; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.UpdateBuilder; import org.apache.cassandra.Util; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.RowUpdateBuilder; -import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.Slices; import org.apache.cassandra.db.compaction.OperationType; +import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; -import org.apache.cassandra.db.lifecycle.Tracker; -import org.apache.cassandra.db.marshal.AsciiType; -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; 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.format.SSTableReader; -import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.io.sstable.format.SSTableReadsListener; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.utils.ByteBufferUtil; -import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Pair; -import static org.apache.cassandra.io.util.DataInputPlus.*; -import static org.junit.Assert.*; +import static org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; public class BigTableBlockWriterTest { @@ -80,7 +73,9 @@ public class BigTableBlockWriterTest public static final String CF_INDEXED = "Indexed1"; public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval"; - private static ColumnFamilyStore cfs; + public static SSTableReader sstable; + public static ColumnFamilyStore store; + private static int expectedRowCount; @BeforeClass public static void defineSchema() throws Exception @@ -95,17 +90,13 @@ public static void defineSchema() throws Exception .minIndexInterval(8) .maxIndexInterval(256) .caching(CachingParams.CACHE_NOTHING)); - } - @Test - public void writeDataFile() - { String ks = KEYSPACE1; String cf = "Standard1"; // clear and create just one sstable for this test Keyspace keyspace = Keyspace.open(ks); - ColumnFamilyStore store = keyspace.getColumnFamilyStore(cf); + store = keyspace.getColumnFamilyStore(cf); store.clearUnsafe(); store.disableAutoCompaction(); @@ -126,10 +117,33 @@ public void writeDataFile() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build() .applyUnsafe(); + expectedRowCount++; } store.forceBlockingFlush(); - SSTableReader sstable = store.getLiveSSTables().iterator().next(); + sstable = store.getLiveSSTables().iterator().next(); + } + + @Test + public void writeDataFile_DataInputPlus() + { + writeDataTestCycle(buffer -> new DataInputStreamPlus(new ByteArrayInputStream(buffer.array()))); + } + + @Test + public void writeDataFile_RebufferingByteBufDataInputPlus() + { + writeDataTestCycle(buffer -> { + EmbeddedChannel channel = new EmbeddedChannel(); + RebufferingByteBufDataInputPlus inputPlus = new RebufferingByteBufDataInputPlus(1 << 10, 1 << 20, channel.config()); + inputPlus.append(Unpooled.wrappedBuffer(buffer)); + return inputPlus; + }); + } + + + private void writeDataTestCycle(Function bufferMapper) + { File dir = store.getDirectories().getDirectoryForNewSSTables(); Descriptor desc = store.newSSTableDescriptor(dir); TableMetadataRef metadata = Schema.instance.getTableMetadataRef(desc); @@ -144,53 +158,54 @@ public void writeDataFile() { if (Files.exists(Paths.get(desc.filenameFor(component)))) { - Pair pair = getSSTableComponentData(sstable, component); + Pair pair = getSSTableComponentData(sstable, component, bufferMapper); + btbw.writeComponent(component.type, pair.left, pair.right); } } Collection readers = btbw.finish(true); - for (SSTableReader reader : readers) - { - System.out.printf("File: %s Generation: %s\n", reader.descriptor.filenameFor(Component.DATA), - reader.descriptor.generation); - } - SSTableReader reader = readers.toArray(new SSTableReader[0])[0]; assertNotEquals(sstable.getFilename(), reader.getFilename()); assertEquals(sstable.estimatedKeys(), reader.estimatedKeys()); assertEquals(sstable.isPendingRepair(), reader.isPendingRepair()); + + assertRowCount(expectedRowCount); } - private Pair getSSTableComponentData(SSTableReader sstable, Component component) + private void assertRowCount(int expected) { - FileHandle componentFile = new FileHandle.Builder(sstable.descriptor.filenameFor(component)) - .bufferSize(1024).complete(); + int count = 0; + for (int i = 0; i < store.metadata().params.minIndexInterval; i++) + { + DecoratedKey dk = Util.dk(String.valueOf(i)); + UnfilteredRowIterator rowIter = sstable.iterator(dk, + Slices.ALL, + ColumnFilter.all(store.metadata()), + false, + SSTableReadsListener.NOOP_LISTENER); + while (rowIter.hasNext()) + { + rowIter.next(); + count++; + } + } + assertEquals(expected, count); + } + private Pair getSSTableComponentData(SSTableReader sstable, Component component, + Function bufferMapper) + { + FileHandle componentFile = new FileHandle.Builder(sstable.descriptor.filenameFor(component)) + .bufferSize(1024).complete(); ByteBuffer buffer = ByteBuffer.allocate((int) componentFile.channel.size()); componentFile.channel.read(buffer, 0); - DataInputStreamPlus is = new DataInputStreamPlus(new ByteArrayInputStream(buffer.array())); - - return Pair.create(is, componentFile.channel.size()); - } + buffer.flip(); - public static ByteBuffer random(int i, int size) - { - byte[] bytes = new byte[size + 4]; - ThreadLocalRandom.current().nextBytes(bytes); - ByteBuffer r = ByteBuffer.wrap(bytes); - r.putInt(0, i); - return r; - } + DataInputPlus inputPlus = bufferMapper.apply(buffer); - public static void truncate(ColumnFamilyStore cfs) - { - cfs.truncateBlocking(); - LifecycleTransaction.waitForDeletions(); - Uninterruptibles.sleepUninterruptibly(10L, TimeUnit.MILLISECONDS); - assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount()); - assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount()); + return Pair.create(inputPlus, componentFile.channel.size()); } } \ No newline at end of file From 7c6c508cca5ba5471ce5616add7db0fcd5f99156 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 17:55:39 -0700 Subject: [PATCH 20/64] Fix bug in size calculation during benchmark --- .../cassandra/test/microbench/ZeroCopyStreamingBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index 3cd7e46cbb11..81167e04e576 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -213,7 +213,7 @@ private EmbeddedChannel createMockNettyChannel(ByteBuf serializedFile) throws Ex public int write(ByteBuffer src) throws IOException { - int size = src.limit(); + int size = src.remaining(); serializedFile.writeBytes(src); return size; } From 6b37a020e38865f2168ca8693e7744b90fe50897 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 11 Jul 2018 18:37:20 -0700 Subject: [PATCH 21/64] Clean up unused imports --- src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java | 1 - .../apache/cassandra/db/streaming/CassandraStreamManager.java | 1 - .../cassandra/io/sstable/format/big/BigTableBlockWriter.java | 1 - .../apache/cassandra/io/util/BufferedDataOutputStreamPlus.java | 2 -- src/java/org/apache/cassandra/io/util/DataOutputPlus.java | 2 -- .../org/apache/cassandra/net/async/ByteBufDataOutputPlus.java | 2 -- .../io/sstable/format/big/BigTableBlockWriterTest.java | 3 --- 7 files changed, 12 deletions(-) diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java index 22467dddb91c..f5423d60892d 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LogReplicaSet.java @@ -26,7 +26,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; import java.util.stream.Collectors; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java index 5d3c51f582b2..43667d0610f9 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamManager.java @@ -47,7 +47,6 @@ import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.TableStreamManager; import org.apache.cassandra.streaming.messages.StreamMessageHeader; -import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.concurrent.Ref; import org.apache.cassandra.utils.concurrent.Refs; diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java index 53f3f146bdd2..398dc7e1ed09 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java index aa0e30717c72..56d88f7c2b3f 100644 --- a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java @@ -24,11 +24,9 @@ import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import net.nicoulaj.compilecommand.annotations.DontInline; - import org.apache.cassandra.config.Config; import org.apache.cassandra.utils.memory.MemoryUtil; import org.apache.cassandra.utils.vint.VIntCoding; diff --git a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java index 35abb6f29c5a..16be42f889be 100644 --- a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java +++ b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java @@ -22,8 +22,6 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import com.google.common.base.Function; - import org.apache.cassandra.utils.vint.VIntCoding; /** diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java index 5501c0bf4854..a77cb0713e84 100644 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java +++ b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java @@ -22,8 +22,6 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import com.google.common.base.Function; - import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufOutputStream; import org.apache.cassandra.io.util.CheckedFunction; diff --git a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java index 3b5571065688..a5c62710e0a0 100644 --- a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java @@ -25,12 +25,9 @@ import java.nio.file.Paths; import java.util.Collection; import java.util.Set; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; import java.util.function.Function; import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.Uninterruptibles; import org.junit.BeforeClass; import org.junit.Test; From 7733e6f041d7d1b80972bd7d227299ea3e977834 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 12:06:11 -0700 Subject: [PATCH 22/64] Add tests for fullyContainedIn logic --- .../db/streaming/CassandraOutgoingFile.java | 7 +- .../streaming/CassandraOutgoingFileTest.java | 67 +++++++++++++++---- 2 files changed, 57 insertions(+), 17 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 8e219fd202eb..7b41305889f0 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -176,7 +176,8 @@ public boolean shouldStreamFullSSTable() return isFullSSTableTransfersEnabled && isFullyContained; } - private boolean fullyContainedIn(List> requestedRanges, SSTableReader sstable) + @VisibleForTesting + public boolean fullyContainedIn(List> requestedRanges, SSTableReader sstable) { if (requestedRanges == null) return false; @@ -185,10 +186,6 @@ private boolean fullyContainedIn(List> requestedRanges, SSTableRead while (iter.hasNext()) { DecoratedKey key = iter.next(); - // todo: this can be made more efficient by sorting and normalizing the ranges (they might already be?) - // todo: then we can use the fact that the tokens we get from the sstable are always increasing, se we don't need - // todo: to compare all tokens to all ranges. (see Verifier.RangeOwnHelper in current trunk for example) - boolean foundFlag = false; for (Range r : requestedRanges) { diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java index 44a885135707..9915c8bf6e2f 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -25,20 +25,21 @@ import org.junit.Test; import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.RowUpdateBuilder; import org.apache.cassandra.db.compaction.CompactionManager; -import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.KeyIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.utils.ByteBufferUtil; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class CassandraOutgoingFileTest @@ -49,6 +50,7 @@ public class CassandraOutgoingFileTest public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval"; private static SSTableReader sstable; + private static ColumnFamilyStore store; @BeforeClass public static void defineSchemaAndPrepareSSTable() @@ -64,7 +66,7 @@ public static void defineSchemaAndPrepareSSTable() .caching(CachingParams.CACHE_NOTHING)); Keyspace keyspace = Keyspace.open(KEYSPACE); - ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1"); + store = keyspace.getColumnFamilyStore("Standard1"); // insert data and compact to a single sstable CompactionManager.instance.disableAutoCompaction(); @@ -83,20 +85,61 @@ public static void defineSchemaAndPrepareSSTable() } @Test - public void testCompleteRangeTriggersFullStreaming() + public void validateFullyContainedIn_SingleContiguousRange_Succeeds() { - Token minToken = sstable.first.getToken().minValue(); - Token maxToken = sstable.last.getToken(); + List> requestedRanges = Arrays.asList(new Range<>(store.getPartitioner().getMinimumToken(), sstable.last.getToken())); - Range requestedRange = new Range<>(minToken, maxToken); + CassandraOutgoingFile cof = new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.ref(), + sstable.getPositionsForRanges(requestedRanges), + requestedRanges, sstable.estimatedKeys()); - Range.normalize(Arrays.asList(requestedRange)); + assertTrue(cof.fullyContainedIn(requestedRanges, sstable)); + } + + @Test + public void validateFullyContainedIn_PartialOverlap_Fails() + { + List> requestedRanges = Arrays.asList(new Range<>(store.getPartitioner().getMinimumToken(), getTokenAtIndex(2))); + + CassandraOutgoingFile cof = new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.ref(), + sstable.getPositionsForRanges(requestedRanges), + requestedRanges, sstable.estimatedKeys()); + + assertFalse(cof.fullyContainedIn(requestedRanges, sstable)); + } + + @Test + public void validateFullyContainedIn_SplitRange_Succeeds() + { + List> requestedRanges = Arrays.asList(new Range<>(store.getPartitioner().getMinimumToken(), getTokenAtIndex(4)), + new Range<>(getTokenAtIndex(2), getTokenAtIndex(6)), + new Range<>(getTokenAtIndex(5), sstable.last.getToken())); - List sections = sstable.getPositionsForRanges(Arrays.asList(requestedRange)); + CassandraOutgoingFile cof = new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.ref(), + sstable.getPositionsForRanges(requestedRanges), + requestedRanges, sstable.estimatedKeys()); - CassandraOutgoingFile cof = new CassandraOutgoingFile(StreamOperation.BOOTSTRAP, sstable.ref(), sections, - Arrays.asList(requestedRange), sstable.estimatedKeys()); + assertTrue(cof.fullyContainedIn(requestedRanges, sstable)); + } - assertTrue(cof.shouldStreamFullSSTable()); + private DecoratedKey getKeyAtIndex(int i) + { + int count = 0; + DecoratedKey key; + + try (KeyIterator iter = new KeyIterator(sstable.descriptor, sstable.metadata())) + { + do + { + key = iter.next(); + count++; + } while (iter.hasNext() && count < i); + } + return key; + } + + private Token getTokenAtIndex(int i) + { + return getKeyAtIndex(i).getToken(); } } \ No newline at end of file From 32c31b5218743a34b312f2759ecf43880bbb9eb1 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 12:26:21 -0700 Subject: [PATCH 23/64] Fix comment --- .../cassandra/net/async/NonClosingDefaultFileRegion.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java b/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java index 9a0c6ab5e44c..46f0ce162a34 100644 --- a/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java +++ b/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java @@ -25,8 +25,10 @@ /** * Netty's DefaultFileRegion closes the underlying FileChannel as soon as - * the refCnt() for the region closes, this is an implementation of the DFR - * that doesn't close the FileChannel. + * the refCnt() for the region drops to zero, this is an implementation of + * the DefaultFileRegion that doesn't close the FileChannel. + * + * See {@link ByteBufDataOutputStreamPlus} for its usage. */ public class NonClosingDefaultFileRegion extends DefaultFileRegion { From de4793be073f4d903c295b82e6e9900b4980a5dc Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 15:22:01 -0700 Subject: [PATCH 24/64] Update benchmark --- .../ZeroCopyStreamingBenchmark.java | 207 +++++++++++++----- 1 file changed, 153 insertions(+), 54 deletions(-) diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index 81167e04e576..d5a7d0a1329d 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -30,10 +31,10 @@ import java.util.concurrent.TimeUnit; import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelOutboundHandlerAdapter; import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultFileRegion; import io.netty.channel.embedded.EmbeddedChannel; import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.db.ColumnFamilyStore; @@ -44,14 +45,17 @@ import org.apache.cassandra.db.streaming.CassandraBlockStreamReader; import org.apache.cassandra.db.streaming.CassandraBlockStreamWriter; import org.apache.cassandra.db.streaming.CassandraStreamHeader; +import org.apache.cassandra.db.streaming.CassandraStreamReader; +import org.apache.cassandra.db.streaming.CassandraStreamWriter; import org.apache.cassandra.db.streaming.ComponentInfo; import org.apache.cassandra.db.streaming.CompressionInfo; +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.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; -import org.apache.cassandra.net.async.NonClosingDefaultFileRegion; import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; @@ -81,12 +85,18 @@ import static org.apache.cassandra.db.streaming.CassandraOutgoingFile.STREAM_COMPONENTS; +/** + * Please ensure that this benchmark is run with stream_throughput_outbound_megabits_per_sec set to a + * really high value otherwise, throttling will kick in and the results will not be meaningful. + */ @Warmup(iterations = 1, time = 1, timeUnit = TimeUnit.SECONDS) -@Measurement(iterations = 1, time = 1, timeUnit = TimeUnit.SECONDS) +@Measurement(iterations = 10, time = 1, timeUnit = TimeUnit.SECONDS) @Fork(value = 1) @Threads(1) public class ZeroCopyStreamingBenchmark { + static final int STREAM_SIZE = 50 * 1024 * 1024; + @State(Scope.Thread) public static class BenchmarkState { @@ -98,9 +108,69 @@ public static class BenchmarkState private static SSTableReader sstable; private static ColumnFamilyStore store; private StreamSession session; + private CassandraBlockStreamWriter blockStreamWriter; + private ByteBuf serializedBlockStream; + private InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); + private CassandraBlockStreamReader blockStreamReader; + private CassandraStreamWriter partialStreamWriter; + private CassandraStreamReader partialStreamReader; + private ByteBuf serializedPartialStream; @Setup - public void defineSchemaAndPrepareSSTable() + public void setupBenchmark() throws IOException + { + Keyspace keyspace = setupSchemaAndKeySpace(); + store = keyspace.getColumnFamilyStore("Standard1"); + generateData(); + + sstable = store.getLiveSSTables().iterator().next(); + session = setupStreamingSessionForTest(); + blockStreamWriter = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); + + generateSerializedBlockStream(); + + CapturingNettyChannel blockStreamCaptureChannel = new CapturingNettyChannel(STREAM_SIZE); + ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, blockStreamCaptureChannel, 1024 * 1024); + blockStreamWriter.write(out); + serializedBlockStream = blockStreamCaptureChannel.getSerializedStream(); + out.close(); + + + session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, serializedBlockStream.readableBytes())); + + CassandraStreamHeader blockStreamHeader = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, + sstable.estimatedKeys(), Collections.emptyList(), + (CompressionInfo) null, 0, sstable.header.toComponent(), + getStreamableComponents(sstable), true, sstable.first, + sstable.metadata().id); + + + blockStreamReader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, + peer, session.planId(), + 0, 0, 0, + null), blockStreamHeader, session); + + List> requestedRanges = Arrays.asList(new Range<>(sstable.first.minValue().getToken(), sstable.last.getToken())); + partialStreamWriter = new CassandraStreamWriter(sstable, sstable.getPositionsForRanges(requestedRanges), session); + + CapturingNettyChannel partialStreamChannel = new CapturingNettyChannel(STREAM_SIZE); + partialStreamWriter.write(ByteBufDataOutputStreamPlus.create(session, partialStreamChannel, 1024 * 1024)); + serializedPartialStream = partialStreamChannel.getSerializedStream(); + + CassandraStreamHeader partialStreamHeader = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, + sstable.estimatedKeys(), sstable.getPositionsForRanges(requestedRanges), + (CompressionInfo) null, 0, sstable.header.toComponent(), + getStreamableComponents(sstable), false, sstable.first, + sstable.metadata().id); + + partialStreamReader = new CassandraStreamReader(new StreamMessageHeader(sstable.metadata().id, + peer, session.planId(), + 0, 0, 0, + null), + partialStreamHeader, session); + } + + private Keyspace setupSchemaAndKeySpace() { SchemaLoader.prepareServer(); SchemaLoader.createKeyspace(KEYSPACE, @@ -112,9 +182,11 @@ public void defineSchemaAndPrepareSSTable() .maxIndexInterval(256) .caching(CachingParams.CACHE_NOTHING)); - Keyspace keyspace = Keyspace.open(KEYSPACE); - store = keyspace.getColumnFamilyStore("Standard1"); + return Keyspace.open(KEYSPACE); + } + private void generateData() + { // insert data and compact to a single sstable CompactionManager.instance.disableAutoCompaction(); for (int j = 0; j < 1_000_000; j++) @@ -127,9 +199,10 @@ public void defineSchemaAndPrepareSSTable() } store.forceBlockingFlush(); CompactionManager.instance.performMaximal(store, false); + } - sstable = store.getLiveSSTables().iterator().next(); - session = setupStreamingSessionForTest(); + private void generateSerializedBlockStream() throws IOException + { } @TearDown @@ -153,41 +226,52 @@ private StreamSession setupStreamingSessionForTest() } } - static final int STREAM_SIZE = 80 * 1024 * 1024; - - @Benchmark @BenchmarkMode(Mode.Throughput) - public void stream(BenchmarkState state) throws Exception + public void blockStreamWriter(BenchmarkState state) throws Exception { - StreamSession session = state.session; - SSTableReader sstable = state.sstable; - InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); - - CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); - - // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed - ByteBuf serializedFile = Unpooled.buffer(STREAM_SIZE); - EmbeddedChannel channel = createMockNettyChannel(serializedFile); - ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); - - writer.write(out); - - session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, serializedFile.readableBytes())); - - CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, - sstable.estimatedKeys(), Collections.emptyList(), - (CompressionInfo) null, 0, sstable.header.toComponent(), - getStreamableComponents(sstable), true, sstable.first, - sstable.metadata().id); + EmbeddedChannel channel = createMockNettyChannel(); + ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(state.session, channel, 1024 * 1024); + state.blockStreamWriter.write(out); + out.close(); + channel.finishAndReleaseAll(); + } + @Benchmark + @BenchmarkMode(Mode.Throughput) + public void blockStreamReader(BenchmarkState state) throws Exception + { + EmbeddedChannel channel = createMockNettyChannel(); + RebufferingByteBufDataInputPlus in = new RebufferingByteBufDataInputPlus(STREAM_SIZE, STREAM_SIZE, channel.config()); + in.append(state.serializedBlockStream.retainedDuplicate()); + SSTableMultiWriter sstableWriter = state.blockStreamReader.read(in); + Collection newSstables = sstableWriter.finished(); + in.close(); + channel.finishAndReleaseAll(); + } - CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); + @Benchmark + @BenchmarkMode(Mode.Throughput) + public void partialStreamWriter(BenchmarkState state) throws Exception + { + EmbeddedChannel channel = createMockNettyChannel(); + ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(state.session, channel, 1024 * 1024); + state.partialStreamWriter.write(out); + out.close(); + channel.finishAndReleaseAll(); + } + @Benchmark + @BenchmarkMode(Mode.Throughput) + public void partialStreamReader(BenchmarkState state) throws Exception + { + EmbeddedChannel channel = createMockNettyChannel(); RebufferingByteBufDataInputPlus in = new RebufferingByteBufDataInputPlus(STREAM_SIZE, STREAM_SIZE, channel.config()); - in.append(serializedFile); - SSTableMultiWriter sstableWriter = reader.read(in); + in.append(state.serializedPartialStream.retainedDuplicate()); + SSTableMultiWriter sstableWriter = state.partialStreamReader.read(in); Collection newSstables = sstableWriter.finished(); + in.close(); + channel.finishAndReleaseAll(); } private static List getStreamableComponents(SSTableReader sstable) @@ -205,42 +289,57 @@ private static List getStreamableComponents(SSTableReader sstable return result; } - private EmbeddedChannel createMockNettyChannel(ByteBuf serializedFile) throws Exception + private EmbeddedChannel createMockNettyChannel() { - WritableByteChannel wbc = new WritableByteChannel() - { - private boolean isOpen = true; + EmbeddedChannel channel = new EmbeddedChannel(); + channel.config().setWriteBufferHighWaterMark(STREAM_SIZE); // avoid blocking + return channel; + } + private static class CapturingNettyChannel extends EmbeddedChannel + { + private final ByteBuf serializedStream; + private final WritableByteChannel proxyWBC = new WritableByteChannel() + { public int write(ByteBuffer src) throws IOException { - int size = src.remaining(); - serializedFile.writeBytes(src); - return size; + int rem = src.remaining(); + serializedStream.writeBytes(src); + return rem; } public boolean isOpen() { - return isOpen; + return true; } public void close() throws IOException { - isOpen = false; } }; - EmbeddedChannel channel = new EmbeddedChannel(new ChannelOutboundHandlerAdapter() + public CapturingNettyChannel(int capacity) { - @Override - public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception + this.serializedStream = alloc().buffer(capacity); + this.pipeline().addLast(new ChannelOutboundHandlerAdapter() { - ((NonClosingDefaultFileRegion) msg).transferTo(wbc, 0); - super.write(ctx, msg, promise); - } - }); - - channel.config().setWriteBufferHighWaterMark(STREAM_SIZE); // avoid rate limiting + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception + { + if (msg instanceof ByteBuf) + serializedStream.writeBytes((ByteBuf) msg); + else if (msg instanceof ByteBuffer) + serializedStream.writeBytes((ByteBuffer) msg); + else if (msg instanceof DefaultFileRegion) + ((DefaultFileRegion) msg).transferTo(proxyWBC, 0); + } + }); + config().setWriteBufferHighWaterMark(capacity); + } - return channel; + public ByteBuf getSerializedStream() + { + return serializedStream.copy(); + } } } From 994840855851da0723bfbce195bd63d2a03436e0 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 16:39:26 -0700 Subject: [PATCH 25/64] Clean up --- src/java/org/apache/cassandra/config/DatabaseDescriptor.java | 2 +- src/java/org/apache/cassandra/db/DiskBoundaries.java | 2 +- .../cassandra/db/streaming/CassandraBlockStreamReader.java | 1 - .../apache/cassandra/db/streaming/CassandraIncomingFile.java | 3 --- .../apache/cassandra/db/streaming/CassandraOutgoingFile.java | 2 -- .../apache/cassandra/db/streaming/CassandraStreamHeader.java | 3 ++- src/java/org/apache/cassandra/utils/Collectors3.java | 2 +- test/conf/cassandra.yaml | 3 ++- .../cassandra/db/streaming/CassandraBlockStreamWriterTest.java | 2 +- .../cassandra/db/streaming/CassandraOutgoingFileTest.java | 2 +- .../org/apache/cassandra/db/streaming/ComponentInfoTest.java | 2 +- test/unit/org/apache/cassandra/io/sstable/ComponentTest.java | 2 +- .../io/sstable/format/big/BigTableBlockWriterTest.java | 2 +- 13 files changed, 12 insertions(+), 16 deletions(-) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 97d07b7a0732..f0d7e89d6207 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2264,7 +2264,7 @@ public static boolean isFullSSTableTransfersEnabled() { if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) { - logger.debug("Internode encryption enabled. Disabling zero copy ssl table transfers for streaming."); + logger.debug("Internode encryption enabled. Disabling zero copy SSTable transfers for streaming."); return false; } else diff --git a/src/java/org/apache/cassandra/db/DiskBoundaries.java b/src/java/org/apache/cassandra/db/DiskBoundaries.java index 5b548fc80be9..22f17b0b6447 100644 --- a/src/java/org/apache/cassandra/db/DiskBoundaries.java +++ b/src/java/org/apache/cassandra/db/DiskBoundaries.java @@ -144,4 +144,4 @@ private int getDiskIndex(DecoratedKey key) assert pos < 0; return -pos - 1; } -} \ No newline at end of file +} diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index 7a731be9fa0d..a9deb9c14366 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -89,7 +89,6 @@ public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHea * @return SSTable transferred * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails. */ - @SuppressWarnings("resource") // input needs to remain open, streams on top of it can't be closed @Override public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java index 02fe68dbe12e..040a1ff7e246 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java @@ -60,11 +60,9 @@ public CassandraIncomingFile(ColumnFamilyStore cfs, StreamSession session, Strea public synchronized void read(DataInputPlus in, int version) throws IOException { CassandraStreamHeader streamHeader = CassandraStreamHeader.serializer.deserialize(in, version); - logger.debug("Incoming stream fullStream={} components={}", streamHeader.fullStream, streamHeader.components); IStreamReader reader; - if (streamHeader.fullStream) { reader = new CassandraBlockStreamReader(header, streamHeader, session); @@ -75,7 +73,6 @@ public synchronized void read(DataInputPlus in, int version) throws IOException ? new CassandraStreamReader(header, streamHeader, session) : new CompressedCassandraStreamReader(header, streamHeader, session); } - size = streamHeader.size(); sstable = reader.read(in); } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 7b41305889f0..cf70befb947b 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -154,7 +154,6 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) out.flush(); IStreamWriter writer; - if (shouldStreamFullSSTable()) { writer = new CassandraBlockStreamWriter(sstable, session, components); @@ -166,7 +165,6 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) new CompressedCassandraStreamWriter(sstable, header.sections, header.compressionInfo, session); } - writer.write(out); } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index aa3474c35362..45f5bd689845 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -66,7 +66,6 @@ public class CassandraStreamHeader /* first token of the sstable required for faster streaming */ public final DecoratedKey firstKey; public final TableId tableId; - public final List components; /* cached size value */ @@ -184,6 +183,8 @@ public String toString() ", sstableLevel=" + sstableLevel + ", header=" + header + ", fullStream=" + fullStream + + ", firstKey=" + firstKey + + ", tableId=" + tableId + ", components=" + components + '}'; } diff --git a/src/java/org/apache/cassandra/utils/Collectors3.java b/src/java/org/apache/cassandra/utils/Collectors3.java index faad7ee94d1b..f8f262e69e65 100644 --- a/src/java/org/apache/cassandra/utils/Collectors3.java +++ b/src/java/org/apache/cassandra/utils/Collectors3.java @@ -51,4 +51,4 @@ public class Collectors3 ImmutableSet.Builder::build, SET_CHARACTERISTICS); } -} \ No newline at end of file +} diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml index b072730877ce..974f736aea12 100644 --- a/test/conf/cassandra.yaml +++ b/test/conf/cassandra.yaml @@ -46,4 +46,5 @@ enable_user_defined_functions: true enable_scripted_user_defined_functions: true prepared_statements_cache_size_mb: 1 corrupted_tombstone_strategy: exception -streaming_full_sstable_transfers_enabled: true \ No newline at end of file +streaming_full_sstable_transfers_enabled: true +stream_throughput_outbound_megabits_per_sec: 200000000 diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java index 158f26924c7e..cb0735ad308e 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java @@ -216,4 +216,4 @@ private StreamSession setupStreamingSessionForTest() session.init(future); return session; } -} \ No newline at end of file +} diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java index 9915c8bf6e2f..754d34731160 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -142,4 +142,4 @@ private Token getTokenAtIndex(int i) { return getKeyAtIndex(i).getToken(); } -} \ No newline at end of file +} diff --git a/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java b/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java index 79f2424fed58..96c441f1a33d 100644 --- a/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java @@ -59,4 +59,4 @@ public void testSerialization_FailsOnBadBytes() throws IOException assertNotEquals(expected, actual); } -} \ No newline at end of file +} diff --git a/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java b/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java index a6a7c3b666ba..a094d3043ace 100644 --- a/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java @@ -38,4 +38,4 @@ public void testTypeCreationFromRepresentation_InvalidIds() assertTrue(Component.Type.fromRepresentation((byte) 11) == Component.Type.CUSTOM); assertTrue(Component.Type.fromRepresentation((byte) 12) == Component.Type.CUSTOM); } -} \ No newline at end of file +} diff --git a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java index a5c62710e0a0..a1a3a5eed31e 100644 --- a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriterTest.java @@ -205,4 +205,4 @@ private Pair getSSTableComponentData(SSTableReader sstable, return Pair.create(inputPlus, componentFile.channel.size()); } -} \ No newline at end of file +} From 762ecbe1745e87d509cfe5deffaecd44c66d9ce5 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 16:59:59 -0700 Subject: [PATCH 26/64] Add documentation to cassandra.yaml --- conf/cassandra.yaml | 9 +++++++++ src/java/org/apache/cassandra/config/Config.java | 2 +- .../org/apache/cassandra/config/DatabaseDescriptor.java | 4 ++-- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 7ff056df5a05..a0d430ee1ea7 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -784,6 +784,15 @@ compaction_throughput_mb_per_sec: 16 # between the sstables, reducing page cache churn and keeping hot rows hot sstable_preemptive_open_interval_in_mb: 50 +# When enabled, permits Cassandra to zero copy eligible SSTables between +# nodes. This speeds up the network transfer significantly subject to +# throttling specified by stream_throughput_outbound_megabits_per_sec. +# Enabling this will reduce the GC pressure on sending and receiving node. +# When unset, the default is enabled. While this feature tries to keep the +# disks balanced, it cannot guarantee it. This feature will be automatically +# disabled if internode encryption is enabled. +# streaming_zerocopy_sstables_enabled: false + # Throttles all outbound streaming file transfers on this node to the # given total throughput in Mbps. This is necessary because Cassandra does # mostly sequential IO when streaming data during bootstrap or repair, which diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 30d8c1e7b56d..ba9214fbf834 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -380,7 +380,7 @@ public class Config public int block_for_peers_timeout_in_secs = 10; public volatile boolean automatic_sstable_upgrade = false; public volatile int max_concurrent_automatic_sstable_upgrades = 1; - public boolean streaming_full_sstable_transfers_enabled = true; + public boolean streaming_zerocopy_sstables_enabled = true; public volatile AuditLogOptions audit_logging_options = new AuditLogOptions(); diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index f0d7e89d6207..ab10112d5749 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2269,8 +2269,8 @@ public static boolean isFullSSTableTransfersEnabled() } else { - return Boolean.valueOf(System.getProperty(Config.PROPERTY_PREFIX + "streaming.enableFullSSTableTransfers", - Boolean.toString(conf.streaming_full_sstable_transfers_enabled))); + return Boolean.valueOf(System.getProperty(Config.PROPERTY_PREFIX + "streaming.enableZeroCopySSTableTransfers", + Boolean.toString(conf.streaming_zerocopy_sstables_enabled))); } } From cf549a4785d3c648174b006198d98a1900e86a77 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 17:01:24 -0700 Subject: [PATCH 27/64] Update CHANGES.txt --- CHANGES.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.txt b/CHANGES.txt index 491792e84a61..7260c2b88d0a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Optimize streaming path in Cassandra (CASSANDRA-14556) * Bump the hints messaging version to match the current one (CASSANDRA-14536) * OffsetAwareConfigurationLoader doesn't set ssl storage port causing bind errors in CircleCI (CASSANDRA-14546) * Report why native_transport_port fails to bind (CASSANDRA-14544) From 306ca09742ba9d4af2356d8e9b817bd3ce09af79 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 17:05:18 -0700 Subject: [PATCH 28/64] Update NEWS.txt --- NEWS.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/NEWS.txt b/NEWS.txt index e1e76dec395c..41351a4bb080 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -86,6 +86,9 @@ New features statements after the cache has been refreshed. CASSANDRA-13985 - Support for audit logging of database activity. If enabled, logs every incoming CQL command request, Authentication (successful as well as unsuccessful login) to a node. + - Faster Streaming using ZeroCopy APIs. If enabled, Cassandra will use ZeroCopy for eligible + SSTables significantly speeding up transfers. Any streaming related operations will see + corresponding improvement. See CASSANDRA-14556. Upgrading --------- From c5bf8e3c225e75870ade269fb09b425f72225d3f Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 12 Jul 2018 18:40:37 -0700 Subject: [PATCH 29/64] Fix typo --- test/conf/cassandra.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml index 974f736aea12..29baf5627eec 100644 --- a/test/conf/cassandra.yaml +++ b/test/conf/cassandra.yaml @@ -46,5 +46,5 @@ enable_user_defined_functions: true enable_scripted_user_defined_functions: true prepared_statements_cache_size_mb: 1 corrupted_tombstone_strategy: exception -streaming_full_sstable_transfers_enabled: true +streaming_zerocopy_sstables_enabled: true stream_throughput_outbound_megabits_per_sec: 200000000 From f9573ccef63535b787e91e46fa886f3e3ff2eee8 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 13 Jul 2018 15:17:19 -0700 Subject: [PATCH 30/64] Use variable length encoding for DecoratedKeys --- .../cassandra/db/streaming/CassandraStreamHeader.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index 45f5bd689845..a1df876863ff 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -244,7 +244,7 @@ public void serialize(CassandraStreamHeader header, DataOutputPlus out, int vers for (ComponentInfo info : header.components) ComponentInfo.serializer.serialize(info, out, version); - ByteBufferUtil.writeWithShortLength(header.firstKey.getKey(), out); + ByteBufferUtil.writeWithVIntLength(header.firstKey.getKey(), out); } } @@ -288,7 +288,7 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function for (int i=0; i < ncomp; i++) components.add(ComponentInfo.serializer.deserialize(in, version)); - ByteBuffer keyBuf = ByteBufferUtil.readWithShortLength(in); + ByteBuffer keyBuf = ByteBufferUtil.readWithVIntLength(in); IPartitioner partitioner = partitionerMapper.apply(tableId); if (partitioner == null) @@ -327,7 +327,7 @@ public long serializedSize(CassandraStreamHeader header, int version) for (ComponentInfo info : header.components) size += ComponentInfo.serializer.serializedSize(info, version); - size += ByteBufferUtil.serializedSizeWithShortLength(header.firstKey.getKey()); + size += ByteBufferUtil.serializedSizeWithVIntLength(header.firstKey.getKey()); } return size; From df29c57729be984d050b267a651ca1795c7338aa Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 13 Jul 2018 15:56:25 -0700 Subject: [PATCH 31/64] Clean up code --- .../test/microbench/ZeroCopyStreamingBenchmark.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index d5a7d0a1329d..4bb22669dbcc 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -127,15 +127,12 @@ public void setupBenchmark() throws IOException session = setupStreamingSessionForTest(); blockStreamWriter = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); - generateSerializedBlockStream(); - CapturingNettyChannel blockStreamCaptureChannel = new CapturingNettyChannel(STREAM_SIZE); ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, blockStreamCaptureChannel, 1024 * 1024); blockStreamWriter.write(out); serializedBlockStream = blockStreamCaptureChannel.getSerializedStream(); out.close(); - session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, serializedBlockStream.readableBytes())); CassandraStreamHeader blockStreamHeader = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, @@ -144,7 +141,6 @@ public void setupBenchmark() throws IOException getStreamableComponents(sstable), true, sstable.first, sstable.metadata().id); - blockStreamReader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, @@ -201,10 +197,6 @@ private void generateData() CompactionManager.instance.performMaximal(store, false); } - private void generateSerializedBlockStream() throws IOException - { - } - @TearDown public void tearDown() throws IOException { From 65c6f504327fd11925b388279d9c5bf8919a344c Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 13 Jul 2018 16:03:07 -0700 Subject: [PATCH 32/64] Throw AssertionError if reBuffer fails --- .../cassandra/net/async/RebufferingByteBufDataInputPlus.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java index 2f8b22843d35..4e667da83788 100644 --- a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java +++ b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java @@ -274,8 +274,8 @@ public long consumeUntil(BufferedDataOutputStreamPlus writer, long len) throws I { throw new EOFException("EOF after " + copied + " bytes out of " + len); } - if (buffer.remaining() == 0) - return copied == 0 ? -1 : copied; + if (buffer.remaining() == 0 && copied < len) + throw new AssertionError("reBuffer() failed to return data"); } int originalLimit = buffer.limit(); From f2e1388305798c14fb08789cd31378ed52d8c9f6 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 13 Jul 2018 16:07:26 -0700 Subject: [PATCH 33/64] Update documentation --- conf/cassandra.yaml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index a0d430ee1ea7..60c10dc4bfc3 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -790,7 +790,10 @@ sstable_preemptive_open_interval_in_mb: 50 # Enabling this will reduce the GC pressure on sending and receiving node. # When unset, the default is enabled. While this feature tries to keep the # disks balanced, it cannot guarantee it. This feature will be automatically -# disabled if internode encryption is enabled. +# disabled if internode encryption is enabled. Compaction strategies such as +# LeveledCompaction will benefit the most while SizeTieredCompaction will +# see gains only in very limited cases. Most strategies will benefit if used +# in combination with CASSANDRA-6696 # streaming_zerocopy_sstables_enabled: false # Throttles all outbound streaming file transfers on this node to the From 690365984e513c31e7864098b29ba87a8be79f4e Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 13 Jul 2018 16:20:34 -0700 Subject: [PATCH 34/64] Use RangeOwnHelper instead of custom logic --- .../db/streaming/CassandraOutgoingFile.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index cf70befb947b..9907b16dea33 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -44,6 +44,8 @@ import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.utils.concurrent.Ref; +import static org.apache.cassandra.db.compaction.Verifier.RangeOwnHelper; + /** * used to transfer the part(or whole) of a SSTable data file */ @@ -175,27 +177,24 @@ public boolean shouldStreamFullSSTable() } @VisibleForTesting - public boolean fullyContainedIn(List> requestedRanges, SSTableReader sstable) + public boolean fullyContainedIn(List> normalizedRanges, SSTableReader sstable) { - if (requestedRanges == null) + if (normalizedRanges == null) return false; + + RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(normalizedRanges); try (KeyIterator iter = new KeyIterator(sstable.descriptor, sstable.metadata())) { while (iter.hasNext()) { DecoratedKey key = iter.next(); - boolean foundFlag = false; - for (Range r : requestedRanges) + try + { + rangeOwnHelper.check(key); + } catch(RuntimeException e) { - if (r.contains(key.getToken())) - { - foundFlag = true; - break; - } - } - - if (foundFlag == false) return false; + } } } return true; From da61f4f1f3d2fa5543ca688e1d71d459dfbd9060 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 24 Jul 2018 11:35:04 -0700 Subject: [PATCH 35/64] change log level to warn --- src/java/org/apache/cassandra/config/DatabaseDescriptor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index ab10112d5749..e2d14d0a101a 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2264,7 +2264,7 @@ public static boolean isFullSSTableTransfersEnabled() { if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) { - logger.debug("Internode encryption enabled. Disabling zero copy SSTable transfers for streaming."); + logger.warn("Internode encryption enabled. Disabling zero copy SSTable transfers for streaming."); return false; } else From bb38bc28d69d1233af97333a7a09fab64799acdb Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 24 Jul 2018 16:58:15 -0700 Subject: [PATCH 36/64] Refactor ComponentInfo -> ComponentManifest --- .../streaming/CassandraBlockStreamReader.java | 44 +++--- .../streaming/CassandraBlockStreamWriter.java | 26 ++-- .../db/streaming/CassandraIncomingFile.java | 2 +- .../db/streaming/CassandraOutgoingFile.java | 19 +-- .../db/streaming/CassandraStreamHeader.java | 71 ++++------ .../cassandra/db/streaming/ComponentInfo.java | 97 ------------- .../db/streaming/ComponentManifest.java | 129 ++++++++++++++++++ .../ZeroCopyStreamingBenchmark.java | 28 +--- .../CassandraBlockStreamWriterTest.java | 21 +-- .../streaming/CassandraStreamHeaderTest.java | 7 +- ...foTest.java => ComponentManifestTest.java} | 13 +- 11 files changed, 211 insertions(+), 246 deletions(-) delete mode 100644 src/java/org/apache/cassandra/db/streaming/ComponentInfo.java create mode 100644 src/java/org/apache/cassandra/db/streaming/ComponentManifest.java rename test/unit/org/apache/cassandra/db/streaming/{ComponentInfoTest.java => ComponentManifestTest.java} (76%) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index a9deb9c14366..d416e2075c64 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; -import java.util.List; import java.util.Set; import com.google.common.base.Throwables; @@ -44,7 +43,6 @@ import org.apache.cassandra.streaming.StreamReceiver; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.messages.StreamMessageHeader; -import org.apache.cassandra.utils.Collectors3; import org.apache.cassandra.utils.FBUtilities; /** @@ -58,7 +56,7 @@ public class CassandraBlockStreamReader implements IStreamReader protected final int sstableLevel; protected final SerializationHeader.Component header; protected final int fileSeqNum; - private final List components; + private final ComponentManifest manifest; private final SSTableFormat.Type format; private final Version version; private final DecoratedKey firstKey; @@ -75,7 +73,7 @@ public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHea } this.session = session; this.tableId = header.tableId; - this.components = streamHeader.components; + this.manifest = streamHeader.componentManifest; this.sstableLevel = streamHeader.sstableLevel; this.header = streamHeader.header; this.format = streamHeader.format; @@ -93,7 +91,7 @@ public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHea @Override public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException { - long totalSize = totalSize(); + long totalSize = manifest.getTotalSize(); ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); @@ -110,23 +108,23 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException try { - Set componentsToWrite = components.stream() - .map(p -> Component.parse(p.type.repr)) - .collect(Collectors3.toImmutableSet()); - - writer = createWriter(cfs, totalSize, componentsToWrite); + writer = createWriter(cfs, totalSize, manifest.getComponents()); long bytesRead = 0; - for (ComponentInfo info : components) + for (Component component : manifest.getComponents()) { + long length = manifest.getSizeForType(component.type); + logger.debug("[Stream #{}] About to receive file {} from {} readBytes = {}, componentSize = {}, totalSize = {}", - session.planId(), info.type, session.peer, FBUtilities.prettyPrintMemory(bytesRead), - FBUtilities.prettyPrintMemory(info.length), FBUtilities.prettyPrintMemory(totalSize)); - writer.writeComponent(info.type, inputPlus, info.length); - session.progress(writer.descriptor.filenameFor(Component.parse(info.type.repr)), ProgressInfo.Direction.IN, info.length, info.length); - bytesRead += info.length; + session.planId(), component, session.peer, FBUtilities.prettyPrintMemory(bytesRead), + FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); + + writer.writeComponent(component.type, inputPlus, length); + session.progress(writer.descriptor.filenameFor(component), ProgressInfo.Direction.IN, length, length); + bytesRead += length; + logger.debug("[Stream #{}] Finished receiving file {} from {} readBytes = {}, componentSize = {}, totalSize = {}", - session.planId(), info.type, session.peer, FBUtilities.prettyPrintMemory(bytesRead), - FBUtilities.prettyPrintMemory(info.length), FBUtilities.prettyPrintMemory(totalSize)); + session.planId(), component, session.peer, FBUtilities.prettyPrintMemory(bytesRead), + FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); } return writer; @@ -170,15 +168,9 @@ protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize Descriptor desc = cfs.newSSTableDescriptor(dataDir, version, format); logger.debug("[Table #{}] {} Components to write - {}", tableId, desc.filenameFor(Component.DATA), componentsToWrite); + + BigTableBlockWriter writer = new BigTableBlockWriter(desc, cfs.metadata, txn, componentsToWrite); return writer; } - - protected long totalSize() - { - long size = 0; - for (ComponentInfo component : components) - size += component.length; - return size; - } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index 1b999095b6fa..ebd7ce9d2d2e 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -45,15 +45,15 @@ public class CassandraBlockStreamWriter implements IStreamWriter private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamWriter.class); protected final SSTableReader sstable; - protected final List components; + protected final ComponentManifest manifest; protected final StreamSession session; private final StreamRateLimiter limiter; - public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, List components) + public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, ComponentManifest manifest) { this.session = session; this.sstable = sstable; - this.components = components; + this.manifest = manifest; this.limiter = StreamManager.getRateLimiter(session.peer); } @@ -67,17 +67,17 @@ public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, @Override public void write(DataOutputStreamPlus output) throws IOException { - long totalSize = totalSize(); + long totalSize = manifest.getTotalSize(); logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}", session.planId(), sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize); long progress = 0L; ByteBufDataOutputStreamPlus byteBufDataOutputStreamPlus = (ByteBufDataOutputStreamPlus) output; - for (ComponentInfo info : components) + for (Component component : manifest.getComponents()) { @SuppressWarnings("resource") // this is closed after the file is transferred by ByteBufDataOutputStreamPlus - FileChannel in = new RandomAccessFile(sstable.descriptor.filenameFor(Component.parse(info.type.repr)), "r").getChannel(); + FileChannel in = new RandomAccessFile(sstable.descriptor.filenameFor(component), "r").getChannel(); // Total Length to transmit for this file long length = in.size(); @@ -85,17 +85,17 @@ public void write(DataOutputStreamPlus output) throws IOException // tracks write progress long bytesRead = 0; logger.debug("[Stream #{}] Block streaming {}.{} gen {} component {} size {}", session.planId(), - sstable.getKeyspaceName(), sstable.getColumnFamilyName(), sstable.descriptor.generation, info.type, length); + sstable.getKeyspaceName(), sstable.getColumnFamilyName(), sstable.descriptor.generation, component, length); bytesRead += byteBufDataOutputStreamPlus.writeToChannel(in, limiter); progress += bytesRead; - session.progress(sstable.descriptor.filenameFor(Component.parse(info.type.repr)), ProgressInfo.Direction.OUT, bytesRead, + session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesRead, length); logger.debug("[Stream #{}] Finished block streaming {}.{} gen {} component {} to {}, xfered = {}, length = {}, totalSize = {}", session.planId(), sstable.getKeyspaceName(), sstable.getColumnFamilyName(), - sstable.descriptor.generation, info.type, session.peer, FBUtilities.prettyPrintMemory(bytesRead), + sstable.descriptor.generation, component, session.peer, FBUtilities.prettyPrintMemory(bytesRead), FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); byteBufDataOutputStreamPlus.flush(); @@ -106,12 +106,4 @@ public void write(DataOutputStreamPlus output) throws IOException } } - - protected long totalSize() - { - long size = 0; - for (ComponentInfo component : components) - size += component.length; - return size; - } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java index 040a1ff7e246..be8504ff2507 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java @@ -60,7 +60,7 @@ public CassandraIncomingFile(ColumnFamilyStore cfs, StreamSession session, Strea public synchronized void read(DataInputPlus in, int version) throws IOException { CassandraStreamHeader streamHeader = CassandraStreamHeader.serializer.deserialize(in, version); - logger.debug("Incoming stream fullStream={} components={}", streamHeader.fullStream, streamHeader.components); + logger.debug("Incoming stream fullStream={} components={}", streamHeader.fullStream, streamHeader.componentManifest); IStreamReader reader; if (streamHeader.fullStream) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 9907b16dea33..f4d657e95334 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -20,8 +20,8 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; +import java.util.LinkedHashMap; import java.util.List; import java.util.Objects; import java.util.UUID; @@ -62,7 +62,7 @@ public class CassandraOutgoingFile implements OutgoingStream private final String filename; private final CassandraStreamHeader header; private final boolean keepSSTableLevel; - private final List components; + private final ComponentManifest manifest; private final boolean isFullyContained; private final List> ranges; @@ -77,7 +77,7 @@ public CassandraOutgoingFile(StreamOperation operation, Ref ref, this.sections = sections; this.ranges = ImmutableList.copyOf(ranges); this.filename = ref.get().getFilename(); - this.components = getComponents(ref.get()); + this.manifest = getComponentManifest(ref.get()); this.isFullyContained = fullyContainedIn(this.ranges, ref.get()); SSTableReader sstable = ref.get(); @@ -88,22 +88,23 @@ public CassandraOutgoingFile(StreamOperation operation, Ref ref, sections, sstable.compression ? sstable.getCompressionMetadata() : null, keepSSTableLevel ? sstable.getSSTableLevel() : 0, - sstable.header.toComponent(), components, shouldStreamFullSSTable(), + sstable.header.toComponent(), manifest, shouldStreamFullSSTable(), sstable.first, sstable.metadata().id); } - private static List getComponents(SSTableReader sstable) + @VisibleForTesting + public static ComponentManifest getComponentManifest(SSTableReader sstable) { - List result = new ArrayList<>(STREAM_COMPONENTS.size()); + LinkedHashMap components = new LinkedHashMap<>(STREAM_COMPONENTS.size()); for (Component component : STREAM_COMPONENTS) { File file = new File(sstable.descriptor.filenameFor(component)); if (file.exists()) - result.add(new ComponentInfo(component.type, file.length())); + components.put(component.type, file.length()); } - return result; + return new ComponentManifest(components); } public static CassandraOutgoingFile fromStream(OutgoingStream stream) @@ -158,7 +159,7 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) IStreamWriter writer; if (shouldStreamFullSSTable()) { - writer = new CassandraBlockStreamWriter(sstable, session, components); + writer = new CassandraBlockStreamWriter(sstable, session, manifest); } else { diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index a1df876863ff..6230aa722973 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -66,7 +66,7 @@ public class CassandraStreamHeader /* first token of the sstable required for faster streaming */ public final DecoratedKey firstKey; public final TableId tableId; - public final List components; + public final ComponentManifest componentManifest; /* cached size value */ private transient final long size; @@ -74,7 +74,7 @@ public class CassandraStreamHeader public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionMetadata compressionMetadata, CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header, - List components, + ComponentManifest componentManifest, boolean fullStream, DecoratedKey firstKey, TableId tableId) { this.version = version; @@ -86,7 +86,7 @@ public CassandraStreamHeader(Version version, SSTableFormat.Type format, long es this.sstableLevel = sstableLevel; this.header = header; this.fullStream = fullStream; - this.components = components; + this.componentManifest = componentManifest; this.firstKey = firstKey; this.tableId = tableId; this.size = calculateSize(); @@ -108,20 +108,20 @@ public CassandraStreamHeader(Version version, SSTableFormat.Type format, long es public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionMetadata compressionMetadata, - int sstableLevel, SerializationHeader.Component header, List components, + int sstableLevel, SerializationHeader.Component header, ComponentManifest componentManifest, boolean fullStream, DecoratedKey firstKey, TableId tableId) { this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header, - components, fullStream, firstKey, tableId); + componentManifest, fullStream, firstKey, tableId); } public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, List sections, CompressionInfo compressionInfo, - int sstableLevel, SerializationHeader.Component header, List components, + int sstableLevel, SerializationHeader.Component header, ComponentManifest componentManifest, boolean fullStream, DecoratedKey firstKey, TableId tableId) { this(version, format, estimatedKeys, sections, null, compressionInfo, sstableLevel, header, - components, fullStream, firstKey, tableId); + componentManifest, fullStream, firstKey, tableId); } public boolean isCompressed() @@ -139,26 +139,20 @@ public long size() private long calculateSize() { - long transferSize = 0; - if (fullStream) + return componentManifest.getTotalSize(); + + long transferSize = 0; + if (compressionInfo != null) { - for (ComponentInfo info : components) - transferSize += info.length; + // calculate total length of transferring chunks + for (CompressionMetadata.Chunk chunk : compressionInfo.chunks) + transferSize += chunk.length + 4; // 4 bytes for CRC } else { - if (compressionInfo != null) - { - // calculate total length of transferring chunks - for (CompressionMetadata.Chunk chunk : compressionInfo.chunks) - transferSize += chunk.length + 4; // 4 bytes for CRC - } - else - { - for (SSTableReader.PartitionPositionBounds section : sections) - transferSize += section.upperPosition - section.lowerPosition; - } + for (SSTableReader.PartitionPositionBounds section : sections) + transferSize += section.upperPosition - section.lowerPosition; } return transferSize; } @@ -166,9 +160,7 @@ private long calculateSize() public synchronized void calculateCompressionInfo() { if (compressionMetadata != null && compressionInfo == null) - { compressionInfo = CompressionInfo.fromCompressionMetadata(compressionMetadata, sections); - } } @Override @@ -185,7 +177,7 @@ public String toString() ", fullStream=" + fullStream + ", firstKey=" + firstKey + ", tableId=" + tableId + - ", components=" + components + + ", componentManifest=" + componentManifest + '}'; } @@ -202,14 +194,14 @@ public boolean equals(Object o) Objects.equals(sections, that.sections) && Objects.equals(compressionInfo, that.compressionInfo) && Objects.equals(header, that.header) && - Objects.equals(components, that.components) && + Objects.equals(componentManifest, that.componentManifest) && Objects.equals(firstKey, that.firstKey) && Objects.equals(tableId, that.tableId); } public int hashCode() { - return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, components, + return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, componentManifest, fullStream, firstKey, tableId); } @@ -240,10 +232,7 @@ public void serialize(CassandraStreamHeader header, DataOutputPlus out, int vers if (header.fullStream) { - out.writeInt(header.components.size()); - for (ComponentInfo info : header.components) - ComponentInfo.serializer.serialize(info, out, version); - + ComponentManifest.serializer.serialize(header.componentManifest, out, version); ByteBufferUtil.writeWithVIntLength(header.firstKey.getKey(), out); } } @@ -277,19 +266,13 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function TableId tableId = TableId.deserialize(in); boolean fullStream = in.readBoolean(); - List components = null; + ComponentManifest manifest = null; DecoratedKey firstKey = null; if (fullStream) { - int ncomp = in.readInt(); - components = new ArrayList<>(ncomp); - - for (int i=0; i < ncomp; i++) - components.add(ComponentInfo.serializer.deserialize(in, version)); - + manifest = ComponentManifest.serializer.deserialize(in, version); ByteBuffer keyBuf = ByteBufferUtil.readWithVIntLength(in); - IPartitioner partitioner = partitionerMapper.apply(tableId); if (partitioner == null) throw new IllegalArgumentException(String.format("Could not determine partitioner for tableId {}", tableId)); @@ -297,7 +280,7 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function } return new CassandraStreamHeader(sstableVersion, format, estimatedKeys, sections, compressionInfo, - sstableLevel, header, components, fullStream, firstKey, tableId); + sstableLevel, header, manifest, fullStream, firstKey, tableId); } public long serializedSize(CassandraStreamHeader header, int version) @@ -323,14 +306,10 @@ public long serializedSize(CassandraStreamHeader header, int version) if (header.fullStream) { - size += TypeSizes.sizeof(header.components.size()); - for (ComponentInfo info : header.components) - size += ComponentInfo.serializer.serializedSize(info, version); - + size += ComponentManifest.serializer.serializedSize(header.componentManifest, version); size += ByteBufferUtil.serializedSizeWithVIntLength(header.firstKey.getKey()); } - return size; } - }; + } } diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentInfo.java b/src/java/org/apache/cassandra/db/streaming/ComponentInfo.java deleted file mode 100644 index 22ee46b9ee7b..000000000000 --- a/src/java/org/apache/cassandra/db/streaming/ComponentInfo.java +++ /dev/null @@ -1,97 +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.streaming; - -import java.io.IOException; - -import org.apache.commons.lang3.builder.EqualsBuilder; -import org.apache.commons.lang3.builder.HashCodeBuilder; -import org.apache.cassandra.db.TypeSizes; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.sstable.Component; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; - -public class ComponentInfo -{ - final Component.Type type; - final long length; - - public ComponentInfo(Component.Type type, long length) - { - assert length >= 0 : "Component length cannot be negative"; - this.type = type; - this.length = length; - } - - @Override - public String toString() - { - return "ComponentInfo{" + - "type=" + type + - ", length=" + length + - '}'; - } - - public boolean equals(Object o) - { - if (this == o) return true; - - if (o == null || getClass() != o.getClass()) return false; - - ComponentInfo that = (ComponentInfo) o; - - return new EqualsBuilder() - .append(length, that.length) - .append(type, that.type) - .isEquals(); - } - - public int hashCode() - { - return new HashCodeBuilder(17, 37) - .append(type) - .append(length) - .toHashCode(); - } - - public static final IVersionedSerializer serializer = new IVersionedSerializer() - { - public void serialize(ComponentInfo info, DataOutputPlus out, int version) throws IOException - { - out.writeByte(info.type.id); - out.writeLong(info.length); - } - - public ComponentInfo deserialize(DataInputPlus in, int version) throws IOException - { - Component.Type type = Component.Type.fromRepresentation(in.readByte()); - long size = in.readLong(); - return new ComponentInfo(type, size); - } - - public long serializedSize(ComponentInfo info, int version) - { - long size = 0; - size += TypeSizes.sizeof(info.type.id); - size += TypeSizes.sizeof(info.length); - return size; - } - }; -} diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java new file mode 100644 index 000000000000..27df8f1d5f3b --- /dev/null +++ b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java @@ -0,0 +1,129 @@ +/* + * 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.streaming; + +import java.io.IOException; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.sstable.Component; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +public class ComponentManifest +{ + private final LinkedHashMap manifest; + private final Set components = new LinkedHashSet<>(Component.Type.values().length); + private final long totalSize; + + public ComponentManifest(Map componentManifest) + { + this.manifest = new LinkedHashMap<>(componentManifest); + + long size = 0; + for (Map.Entry entry : this.manifest.entrySet()) + { + size += entry.getValue(); + this.components.add(Component.parse(entry.getKey().repr)); + } + + this.totalSize = size; + } + + public Long getSizeForType(Component.Type type) + { + return manifest.get(type); + } + + public long getTotalSize() + { + return totalSize; + } + + public Set getComponents() + { + return Collections.unmodifiableSet(components); + } + + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ComponentManifest that = (ComponentManifest) o; + return totalSize == that.totalSize && + Objects.equals(manifest, that.manifest); + } + + public int hashCode() + { + + return Objects.hash(manifest, totalSize); + } + + public static final IVersionedSerializer serializer = new IVersionedSerializer() + { + public void serialize(ComponentManifest manifest, DataOutputPlus out, int version) throws IOException + { + out.writeInt(manifest.manifest.size()); + for (Map.Entry entry : manifest.manifest.entrySet()) + serialize(entry.getKey(), entry.getValue(), out); + } + + public ComponentManifest deserialize(DataInputPlus in, int version) throws IOException + { + LinkedHashMap components = new LinkedHashMap<>(Component.Type.values().length); + + int size = in.readInt(); + assert size >= 0 : "Invalid number of components"; + + for (int i = 0; i < size; i++) + { + Component.Type type = Component.Type.fromRepresentation(in.readByte()); + long length = in.readLong(); + components.put(type, length); + } + + return new ComponentManifest(components); + } + + public long serializedSize(ComponentManifest manifest, int version) + { + long size = 0; + size += TypeSizes.sizeof(manifest.manifest.size()); + for (Map.Entry entry : manifest.manifest.entrySet()) + { + size += TypeSizes.sizeof(entry.getKey().id); + size += TypeSizes.sizeof(entry.getValue()); + } + return size; + } + + private void serialize(Component.Type type, long size, DataOutputPlus out) throws IOException + { + out.writeByte(type.id); + out.writeLong(size); + } + }; +} diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index 4bb22669dbcc..6f627d4b2455 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -18,11 +18,9 @@ package org.apache.cassandra.test.microbench; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -44,14 +42,13 @@ import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.streaming.CassandraBlockStreamReader; import org.apache.cassandra.db.streaming.CassandraBlockStreamWriter; +import org.apache.cassandra.db.streaming.CassandraOutgoingFile; import org.apache.cassandra.db.streaming.CassandraStreamHeader; import org.apache.cassandra.db.streaming.CassandraStreamReader; import org.apache.cassandra.db.streaming.CassandraStreamWriter; -import org.apache.cassandra.db.streaming.ComponentInfo; import org.apache.cassandra.db.streaming.CompressionInfo; 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.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.InetAddressAndPort; @@ -83,8 +80,6 @@ import org.openjdk.jmh.annotations.Threads; import org.openjdk.jmh.annotations.Warmup; -import static org.apache.cassandra.db.streaming.CassandraOutgoingFile.STREAM_COMPONENTS; - /** * Please ensure that this benchmark is run with stream_throughput_outbound_megabits_per_sec set to a * really high value otherwise, throttling will kick in and the results will not be meaningful. @@ -125,7 +120,7 @@ public void setupBenchmark() throws IOException sstable = store.getLiveSSTables().iterator().next(); session = setupStreamingSessionForTest(); - blockStreamWriter = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); + blockStreamWriter = new CassandraBlockStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); CapturingNettyChannel blockStreamCaptureChannel = new CapturingNettyChannel(STREAM_SIZE); ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, blockStreamCaptureChannel, 1024 * 1024); @@ -138,7 +133,7 @@ public void setupBenchmark() throws IOException CassandraStreamHeader blockStreamHeader = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), - getStreamableComponents(sstable), true, sstable.first, + CassandraOutgoingFile.getComponentManifest(sstable), true, sstable.first, sstable.metadata().id); blockStreamReader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, @@ -156,7 +151,7 @@ public void setupBenchmark() throws IOException CassandraStreamHeader partialStreamHeader = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), sstable.getPositionsForRanges(requestedRanges), (CompressionInfo) null, 0, sstable.header.toComponent(), - getStreamableComponents(sstable), false, sstable.first, + CassandraOutgoingFile.getComponentManifest(sstable), false, sstable.first, sstable.metadata().id); partialStreamReader = new CassandraStreamReader(new StreamMessageHeader(sstable.metadata().id, @@ -266,21 +261,6 @@ public void partialStreamReader(BenchmarkState state) throws Exception channel.finishAndReleaseAll(); } - private static List getStreamableComponents(SSTableReader sstable) - { - List result = new ArrayList<>(STREAM_COMPONENTS.size()); - for (Component component : STREAM_COMPONENTS) - { - File file = new File(sstable.descriptor.filenameFor(component)); - if (file.exists()) - { - result.add(new ComponentInfo(component.type, file.length())); - } - } - - return result; - } - private EmbeddedChannel createMockNettyChannel() { EmbeddedChannel channel = new EmbeddedChannel(); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java index cb0735ad308e..4c6bfd92101a 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java @@ -118,7 +118,7 @@ public void testBlockWriterOverWire() throws IOException { StreamSession session = setupStreamingSessionForTest(); - CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); + CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); EmbeddedChannel channel = new EmbeddedChannel(); ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); @@ -135,7 +135,7 @@ public void testBlockReadingAndWritingOverWire() throws Exception StreamSession session = setupStreamingSessionForTest(); InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); - CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, getStreamableComponents(sstable)); + CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed ByteBuf serializedFile = Unpooled.buffer(8192); @@ -146,7 +146,7 @@ public void testBlockReadingAndWritingOverWire() throws Exception session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104)); - CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), getStreamableComponents(sstable), true, sstable.first, sstable.metadata().id); + CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), CassandraOutgoingFile.getComponentManifest(sstable), true, sstable.first, sstable.metadata().id); CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); @@ -189,21 +189,6 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) }); } - private static List getStreamableComponents(SSTableReader sstable) - { - List result = new ArrayList<>(STREAM_COMPONENTS.size()); - for (Component component : STREAM_COMPONENTS) - { - File file = new File(sstable.descriptor.filenameFor(component)); - if (file.exists()) - { - result.add(new ComponentInfo(component.type, file.length())); - } - } - - return result; - } - private StreamSession setupStreamingSessionForTest() { StreamCoordinator streamCoordinator = new StreamCoordinator(StreamOperation.BOOTSTRAP, 1, new DefaultConnectionFactory(), false, null, PreviewKind.NONE); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index d86f17d3d76a..02acc27997d2 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -20,9 +20,12 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.junit.Test; import org.apache.cassandra.cql3.statements.CreateTableStatement; @@ -63,7 +66,7 @@ public void serializerTest_FullSSTableTransfer() String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); - List ci = ImmutableList.of(new ComponentInfo(Component.Type.DATA, 100)); + ComponentManifest manifest = new ComponentManifest(new HashMap(ImmutableMap.of(Component.Type.DATA, 100L))); CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, SSTableFormat.Type.BIG, @@ -72,7 +75,7 @@ public void serializerTest_FullSSTableTransfer() ((CompressionMetadata) null), 0, SerializationHeader.makeWithoutStats(metadata).toComponent(), - ci, true, Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER), metadata.id); + manifest, true, Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER), metadata.id); SerializationUtils.assertSerializationCycle(header, new TestableCassandraStreamHeaderSerializer()); } diff --git a/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java similarity index 76% rename from test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java rename to test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java index 96c441f1a33d..f5874fbdca38 100644 --- a/test/unit/org/apache/cassandra/db/streaming/ComponentInfoTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java @@ -19,6 +19,7 @@ package org.apache.cassandra.db.streaming; import java.io.IOException; +import java.util.HashMap; import org.junit.Test; @@ -34,28 +35,28 @@ import static org.junit.Assert.assertNotEquals; -public class ComponentInfoTest +public class ComponentManifestTest { @Test public void testSerialization() { - ComponentInfo expected = new ComponentInfo(Component.Type.DATA, 100); - SerializationUtils.assertSerializationCycle(expected, ComponentInfo.serializer); + ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.Type.DATA, 100L); }}); + SerializationUtils.assertSerializationCycle(expected, ComponentManifest.serializer); } @Test(expected = AssertionError.class) public void testSerialization_FailsOnBadBytes() throws IOException { ByteBuf buf = Unpooled.buffer(512); - ComponentInfo expected = new ComponentInfo(Component.Type.DATA, 100); + ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.Type.DATA, 100L); }}); DataOutputPlus output = new ByteBufDataOutputPlus(buf); - ComponentInfo.serializer.serialize(expected, output, MessagingService.VERSION_40); + ComponentManifest.serializer.serialize(expected, output, MessagingService.VERSION_40); buf.setInt(0, -100); DataInputPlus input = new ByteBufDataInputPlus(buf); - ComponentInfo actual = ComponentInfo.serializer.deserialize(input, MessagingService.VERSION_40); + ComponentManifest actual = ComponentManifest.serializer.deserialize(input, MessagingService.VERSION_40); assertNotEquals(expected, actual); } From 96469c2efc77eefc93245b3d80d0ba0bfdb73972 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 24 Jul 2018 17:19:48 -0700 Subject: [PATCH 37/64] Augment RangeOwnHelper to return a boolean --- .../cassandra/db/compaction/Verifier.java | 20 +++++++++++++++++-- .../db/streaming/CassandraOutgoingFile.java | 7 +------ 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java index bc9679da40a4..4d5b8e3f40bb 100644 --- a/src/java/org/apache/cassandra/db/compaction/Verifier.java +++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java @@ -361,12 +361,26 @@ public RangeOwnHelper(List> normalizedRanges) * @throws RuntimeException if the key is not contained */ public void check(DecoratedKey key) + { + if (!checkBoolean(key)) + throw new RuntimeException("Key " + key + " is not contained in the given ranges"); + } + + /** + * check if the given key is contained in any of the given ranges + * + * Must be called in sorted order - key should be increasing + * + * @param key the key + * @return boolean + */ + public boolean checkBoolean(DecoratedKey key) { assert lastKey == null || key.compareTo(lastKey) > 0; lastKey = key; if (normalizedRanges.isEmpty()) // handle tests etc where we don't have any ranges - return; + return true; if (rangeIndex > normalizedRanges.size() - 1) throw new IllegalStateException("RangeOwnHelper can only be used to find the first out-of-range-token"); @@ -375,8 +389,10 @@ public void check(DecoratedKey key) { rangeIndex++; if (rangeIndex > normalizedRanges.size() - 1) - throw new RuntimeException("Key "+key+" is not contained in the given ranges"); + return false; } + + return true; } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index f4d657e95334..085131789942 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -189,13 +189,8 @@ public boolean fullyContainedIn(List> normalizedRanges, SSTableRead while (iter.hasNext()) { DecoratedKey key = iter.next(); - try - { - rangeOwnHelper.check(key); - } catch(RuntimeException e) - { + if (!rangeOwnHelper.checkBoolean(key)) return false; - } } } return true; From 5f2624d6de9da27137e807ac8904d9ea1a6ea085 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 24 Jul 2018 17:42:36 -0700 Subject: [PATCH 38/64] Make fields final --- .../db/streaming/CassandraBlockStreamReader.java | 10 +++++----- .../db/streaming/CassandraBlockStreamWriter.java | 6 +++--- .../db/streaming/CassandraBlockStreamWriterTest.java | 5 ----- 3 files changed, 8 insertions(+), 13 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index d416e2075c64..0204b7e53ae1 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -51,11 +51,11 @@ public class CassandraBlockStreamReader implements IStreamReader { private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamReader.class); - protected final TableId tableId; - protected final StreamSession session; - protected final int sstableLevel; - protected final SerializationHeader.Component header; - protected final int fileSeqNum; + private final TableId tableId; + private final StreamSession session; + private final int sstableLevel; + private final SerializationHeader.Component header; + private final int fileSeqNum; private final ComponentManifest manifest; private final SSTableFormat.Type format; private final Version version; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index ebd7ce9d2d2e..ffd92b4bd4f7 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -44,9 +44,9 @@ public class CassandraBlockStreamWriter implements IStreamWriter { private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamWriter.class); - protected final SSTableReader sstable; - protected final ComponentManifest manifest; - protected final StreamSession session; + private final SSTableReader sstable; + private final ComponentManifest manifest; + private final StreamSession session; private final StreamRateLimiter limiter; public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, ComponentManifest manifest) diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java index 4c6bfd92101a..8cd4dc3d2ef2 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java @@ -19,14 +19,11 @@ package org.apache.cassandra.db.streaming; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.List; import java.util.Queue; import java.util.UUID; @@ -45,7 +42,6 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.RowUpdateBuilder; import org.apache.cassandra.db.compaction.CompactionManager; -import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.InetAddressAndPort; @@ -67,7 +63,6 @@ import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.db.streaming.CassandraOutgoingFile.STREAM_COMPONENTS; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; From efa97da9358130c9cdfd6e77f0873f723518cbe3 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Tue, 24 Jul 2018 23:43:54 -0700 Subject: [PATCH 39/64] Limit faster streaming to LCS. Reset level information using CassandraStreamHeader --- .../db/streaming/CassandraBlockStreamReader.java | 1 + .../db/streaming/CassandraOutgoingFile.java | 12 +++++++++++- .../db/streaming/CassandraOutgoingFileTest.java | 2 +- 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index 0204b7e53ae1..94a42ffd1f16 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -127,6 +127,7 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); } + writer.descriptor.getMetadataSerializer().mutateLevel(writer.descriptor, sstableLevel); return writer; } catch (Throwable e) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 085131789942..81b8be7f202e 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -31,7 +31,9 @@ import com.google.common.collect.ImmutableList; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.compaction.LeveledCompactionStrategy; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.Component; @@ -174,7 +176,15 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) @VisibleForTesting public boolean shouldStreamFullSSTable() { - return isFullSSTableTransfersEnabled && isFullyContained; + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(getTableId()); + + if (cfs == null) + return false; + + if (cfs.getCompactionStrategyManager().getCompactionStrategyFor(ref.get()) instanceof LeveledCompactionStrategy) + return isFullSSTableTransfersEnabled && isFullyContained; + + return false; } @VisibleForTesting diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java index 754d34731160..e1811380df2b 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -66,7 +66,7 @@ public static void defineSchemaAndPrepareSSTable() .caching(CachingParams.CACHE_NOTHING)); Keyspace keyspace = Keyspace.open(KEYSPACE); - store = keyspace.getColumnFamilyStore("Standard1"); + store = keyspace.getColumnFamilyStore(CF_STANDARD); // insert data and compact to a single sstable CompactionManager.instance.disableAutoCompaction(); From 0cd0be21e8c654519d18600e9502db3a5cd56ebe Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Wed, 25 Jul 2018 12:17:31 +0100 Subject: [PATCH 40/64] Rename checkBoolean --- .../cassandra/db/compaction/Verifier.java | 10 +++---- .../db/streaming/CassandraOutgoingFile.java | 2 +- .../org/apache/cassandra/db/VerifyTest.java | 30 +++++++++---------- 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java index 4d5b8e3f40bb..db49369e5ddd 100644 --- a/src/java/org/apache/cassandra/db/compaction/Verifier.java +++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java @@ -180,7 +180,7 @@ public void verify() while (iter.hasNext()) { DecoratedKey key = iter.next(); - rangeOwnHelper.check(key); + rangeOwnHelper.validate(key); } } catch (Throwable t) @@ -262,7 +262,7 @@ public void verify() { try { - rangeOwnHelper.check(key); + rangeOwnHelper.validate(key); } catch (Throwable t) { @@ -360,9 +360,9 @@ public RangeOwnHelper(List> normalizedRanges) * @param key the key * @throws RuntimeException if the key is not contained */ - public void check(DecoratedKey key) + public void validate(DecoratedKey key) { - if (!checkBoolean(key)) + if (!check(key)) throw new RuntimeException("Key " + key + " is not contained in the given ranges"); } @@ -374,7 +374,7 @@ public void check(DecoratedKey key) * @param key the key * @return boolean */ - public boolean checkBoolean(DecoratedKey key) + public boolean check(DecoratedKey key) { assert lastKey == null || key.compareTo(lastKey) > 0; lastKey = key; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 81b8be7f202e..fb82daa40271 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -199,7 +199,7 @@ public boolean fullyContainedIn(List> normalizedRanges, SSTableRead while (iter.hasNext()) { DecoratedKey key = iter.next(); - if (!rangeOwnHelper.checkBoolean(key)) + if (!rangeOwnHelper.check(key)) return false; } } diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java index c9dbe1410ebe..063227402637 100644 --- a/test/unit/org/apache/cassandra/db/VerifyTest.java +++ b/test/unit/org/apache/cassandra/db/VerifyTest.java @@ -612,15 +612,15 @@ public void testRangeOwnHelper() Verifier.RangeOwnHelper roh = new Verifier.RangeOwnHelper(normalized); - roh.check(dk(1)); - roh.check(dk(10)); - roh.check(dk(11)); - roh.check(dk(21)); - roh.check(dk(25)); + roh.validate(dk(1)); + roh.validate(dk(10)); + roh.validate(dk(11)); + roh.validate(dk(21)); + roh.validate(dk(25)); boolean gotException = false; try { - roh.check(dk(26)); + roh.validate(dk(26)); } catch (Throwable t) { @@ -635,9 +635,9 @@ public void testRangeOwnHelperBadToken() List> normalized = new ArrayList<>(); normalized.add(r(0,10)); Verifier.RangeOwnHelper roh = new Verifier.RangeOwnHelper(normalized); - roh.check(dk(1)); + roh.validate(dk(1)); // call with smaller token to get exception - roh.check(dk(0)); + roh.validate(dk(0)); } @@ -646,9 +646,9 @@ public void testRangeOwnHelperNormalize() { List> normalized = Range.normalize(Collections.singletonList(r(0,0))); Verifier.RangeOwnHelper roh = new Verifier.RangeOwnHelper(normalized); - roh.check(dk(Long.MIN_VALUE)); - roh.check(dk(0)); - roh.check(dk(Long.MAX_VALUE)); + roh.validate(dk(Long.MIN_VALUE)); + roh.validate(dk(0)); + roh.validate(dk(Long.MAX_VALUE)); } @Test @@ -656,12 +656,12 @@ public void testRangeOwnHelperNormalizeWrap() { List> normalized = Range.normalize(Collections.singletonList(r(Long.MAX_VALUE - 1000,Long.MIN_VALUE + 1000))); Verifier.RangeOwnHelper roh = new Verifier.RangeOwnHelper(normalized); - roh.check(dk(Long.MIN_VALUE)); - roh.check(dk(Long.MAX_VALUE)); + roh.validate(dk(Long.MIN_VALUE)); + roh.validate(dk(Long.MAX_VALUE)); boolean gotException = false; try { - roh.check(dk(26)); + roh.validate(dk(26)); } catch (Throwable t) { @@ -673,7 +673,7 @@ public void testRangeOwnHelperNormalizeWrap() @Test public void testEmptyRanges() { - new Verifier.RangeOwnHelper(Collections.emptyList()).check(dk(1)); + new Verifier.RangeOwnHelper(Collections.emptyList()).validate(dk(1)); } private DecoratedKey dk(long l) From 00c807318ee59ba0b7f7313c01f1a074cd2b5081 Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Wed, 25 Jul 2018 12:33:32 +0100 Subject: [PATCH 41/64] Propagate result of writer.abort(), replaced deprecated Throwables.propagate() call --- .../db/streaming/CassandraBlockStreamReader.java | 9 ++++----- .../io/sstable/format/big/BigTableBlockWriter.java | 6 ++---- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index 94a42ffd1f16..fc8b020e7e0a 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -133,12 +133,11 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException catch (Throwable e) { logger.error("[Stream {}] Error while reading from stream on ks='{}' and table='{}'.", - session.planId(), cfs.keyspace.getName(), cfs.getTableName(), e); + session.planId(), cfs.keyspace.getName(), cfs.getTableName(), e); if (writer != null) - { - writer.abort(e); - } - throw Throwables.propagate(e); + e = writer.abort(e); + Throwables.throwIfUnchecked(e); + throw new RuntimeException(e); } } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java index 398dc7e1ed09..76a658a24205 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java @@ -165,8 +165,7 @@ public TableId getTableId() public Throwable commit(Throwable accumulate) { for (SequentialWriter writer : componentWriters.values()) - writer.commit(accumulate); - + accumulate = writer.commit(accumulate); return accumulate; } @@ -174,8 +173,7 @@ public Throwable commit(Throwable accumulate) public Throwable abort(Throwable accumulate) { for (SequentialWriter writer : componentWriters.values()) - writer.abort(accumulate); - + accumulate = writer.abort(accumulate); return accumulate; } From 699a44a71e73a68a87d4635d0c9aae44113d15b2 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 25 Jul 2018 15:04:04 -0700 Subject: [PATCH 42/64] Use String format specifier --- .../cassandra/db/streaming/CassandraBlockStreamReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index fc8b020e7e0a..b9aae5421793 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -68,7 +68,7 @@ public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHea // we should only ever be streaming pending repair // sstables if the session has a pending repair id if (!session.getPendingRepair().equals(header.pendingRepair)) - throw new IllegalStateException(String.format("Stream Session & SSTable ({}) pendingRepair UUID mismatch.", + throw new IllegalStateException(String.format("Stream Session & SSTable (%s) pendingRepair UUID mismatch.", header.tableId)); } this.session = session; From 412ed670e45528b3c33dcbb552928d1fae741d75 Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Wed, 25 Jul 2018 13:22:26 +0100 Subject: [PATCH 43/64] CBSR: remove redundant fields, make logging consistent (esp. re: prettyPrintMemory() use) --- .../streaming/CassandraBlockStreamReader.java | 98 +++++++++---------- 1 file changed, 48 insertions(+), 50 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index b9aae5421793..d5295d49efdd 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -27,15 +27,11 @@ 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.SerializationHeader; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SSTableMultiWriter; -import org.apache.cassandra.io.sstable.format.SSTableFormat; -import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.sstable.format.big.BigTableBlockWriter; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.schema.TableId; @@ -43,7 +39,10 @@ import org.apache.cassandra.streaming.StreamReceiver; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.messages.StreamMessageHeader; -import org.apache.cassandra.utils.FBUtilities; + +import static java.lang.String.format; + +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; /** * CassandraBlockStreamReader reads SSTable off the wire and writes it to disk. @@ -51,35 +50,25 @@ public class CassandraBlockStreamReader implements IStreamReader { private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamReader.class); + private final TableId tableId; private final StreamSession session; - private final int sstableLevel; - private final SerializationHeader.Component header; - private final int fileSeqNum; - private final ComponentManifest manifest; - private final SSTableFormat.Type format; - private final Version version; - private final DecoratedKey firstKey; - - public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHeader streamHeader, StreamSession session) + private final CassandraStreamHeader header; + private final int fileSequenceNumber; + + public CassandraBlockStreamReader(StreamMessageHeader messageHeader, CassandraStreamHeader streamHeader, StreamSession session) { if (session.getPendingRepair() != null) { - // we should only ever be streaming pending repair - // sstables if the session has a pending repair id - if (!session.getPendingRepair().equals(header.pendingRepair)) - throw new IllegalStateException(String.format("Stream Session & SSTable (%s) pendingRepair UUID mismatch.", - header.tableId)); + // we should only ever be streaming pending repair sstables if the session has a pending repair id + if (!session.getPendingRepair().equals(messageHeader.pendingRepair)) + throw new IllegalStateException(format("Stream Session & SSTable (%s) pendingRepair UUID mismatch.", messageHeader.tableId)); } + + this.header = streamHeader; this.session = session; - this.tableId = header.tableId; - this.manifest = streamHeader.componentManifest; - this.sstableLevel = streamHeader.sstableLevel; - this.header = streamHeader.header; - this.format = streamHeader.format; - this.fileSeqNum = header.sequenceNumber; - this.version = streamHeader.version; - this.firstKey = streamHeader.firstKey; + this.tableId = messageHeader.tableId; + this.fileSequenceNumber = messageHeader.sequenceNumber; } /** @@ -91,18 +80,22 @@ public CassandraBlockStreamReader(StreamMessageHeader header, CassandraStreamHea @Override public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException { - long totalSize = manifest.getTotalSize(); - ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); - if (cfs == null) { // schema was dropped during streaming - throw new IOException("CF " + tableId + " was dropped during streaming"); + throw new IOException("Table " + tableId + " was dropped during streaming"); } - logger.debug("[Stream #{}] Start receiving file #{} from {}, size = {}, ks = '{}', table = '{}'.", - session.planId(), fileSeqNum, session.peer, totalSize, cfs.keyspace.getName(), cfs.getTableName()); + ComponentManifest manifest = header.componentManifest; + long totalSize = manifest.getTotalSize(); + + logger.debug("[Stream #{}] Started receiving sstable #{} from {}, size = {}, table = {}", + session.planId(), + fileSequenceNumber, + session.peer, + prettyPrintMemory(totalSize), + cfs.metadata()); BigTableBlockWriter writer = null; @@ -114,26 +107,33 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException { long length = manifest.getSizeForType(component.type); - logger.debug("[Stream #{}] About to receive file {} from {} readBytes = {}, componentSize = {}, totalSize = {}", - session.planId(), component, session.peer, FBUtilities.prettyPrintMemory(bytesRead), - FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); + logger.debug("[Stream #{}] Started receiving {} component from {}, componentSize = {}, readBytes = {}, totalSize = {}", + session.planId(), + component, + session.peer, + prettyPrintMemory(length), + prettyPrintMemory(bytesRead), + prettyPrintMemory(totalSize)); writer.writeComponent(component.type, inputPlus, length); session.progress(writer.descriptor.filenameFor(component), ProgressInfo.Direction.IN, length, length); bytesRead += length; - logger.debug("[Stream #{}] Finished receiving file {} from {} readBytes = {}, componentSize = {}, totalSize = {}", - session.planId(), component, session.peer, FBUtilities.prettyPrintMemory(bytesRead), - FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); + logger.debug("[Stream #{}] Finished receiving {} component from {}, componentSize = {}, readBytes = {}, totalSize = {}", + session.planId(), + component, + session.peer, + prettyPrintMemory(length), + prettyPrintMemory(bytesRead), + prettyPrintMemory(totalSize)); } - writer.descriptor.getMetadataSerializer().mutateLevel(writer.descriptor, sstableLevel); + writer.descriptor.getMetadataSerializer().mutateLevel(writer.descriptor, header.sstableLevel); return writer; } catch (Throwable e) { - logger.error("[Stream {}] Error while reading from stream on ks='{}' and table='{}'.", - session.planId(), cfs.keyspace.getName(), cfs.getTableName(), e); + logger.error("[Stream {}] Error while reading sstable from stream for table = {}", session.planId(), cfs.metadata(), e); if (writer != null) e = writer.abort(e); Throwables.throwIfUnchecked(e); @@ -145,9 +145,9 @@ private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOExceptio { Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize); if (localDir == null) - throw new IOException(String.format("Insufficient disk space to store %s", FBUtilities.prettyPrintMemory(totalSize))); + throw new IOException(format("Insufficient disk space to store %s", prettyPrintMemory(totalSize))); - File dir = cfs.getDirectories().getLocationForDisk(cfs.getDiskBoundaries().getCorrectDiskForKey(firstKey)); + File dir = cfs.getDirectories().getLocationForDisk(cfs.getDiskBoundaries().getCorrectDiskForKey(header.firstKey)); if (dir == null) return cfs.getDirectories().getDirectoryForNewSSTables(); @@ -156,7 +156,7 @@ private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOExceptio } @SuppressWarnings("resource") - protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize, Set componentsToWrite) throws IOException + protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize, Set components) throws IOException { File dataDir = getDataDir(cfs, totalSize); @@ -165,12 +165,10 @@ protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize LifecycleTransaction txn = CassandraStreamReceiver.fromReceiver(session.getAggregator(tableId)).getTransaction(); - Descriptor desc = cfs.newSSTableDescriptor(dataDir, version, format); - - logger.debug("[Table #{}] {} Components to write - {}", tableId, desc.filenameFor(Component.DATA), componentsToWrite); + Descriptor desc = cfs.newSSTableDescriptor(dataDir, header.version, header.format); + logger.debug("[Table #{}] {} Components to write: {}", cfs.metadata(), desc.filenameFor(Component.DATA), components); - BigTableBlockWriter writer = new BigTableBlockWriter(desc, cfs.metadata, txn, componentsToWrite); - return writer; + return new BigTableBlockWriter(desc, cfs.metadata, txn, components); } } From 2954bb2fa2d3cc9c93908941e55fa3fc348fa022 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 25 Jul 2018 15:43:22 -0700 Subject: [PATCH 44/64] BTBW use static imports, fix assertion logic and reformat --- .../format/big/BigTableBlockWriter.java | 66 +++++++++++-------- 1 file changed, 38 insertions(+), 28 deletions(-) diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java index 76a658a24205..34f296f8a9c6 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java @@ -15,19 +15,18 @@ * 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.File; import java.io.IOException; import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; +import java.util.EnumMap; import java.util.Map; import java.util.Set; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,51 +47,61 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadataRef; +import static java.lang.String.format; +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; + public class BigTableBlockWriter extends SSTable implements SSTableMultiWriter { + private static final Logger logger = LoggerFactory.getLogger(BigTableBlockWriter.class); + private final TableMetadataRef metadata; - private final LifecycleTransaction txn; private volatile SSTableReader finalReader; private final Map componentWriters; - private final Logger logger = LoggerFactory.getLogger(BigTableBlockWriter.class); - - private final SequentialWriterOption writerOption = SequentialWriterOption.newBuilder() - .trickleFsync(false) - .bufferSize(2 * 1024 * 1024) - .bufferType(BufferType.OFF_HEAP) - .build(); - public static final ImmutableSet supportedComponents = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.STATS, - Component.COMPRESSION_INFO, Component.FILTER, Component.SUMMARY, - Component.DIGEST, Component.CRC); + private static final SequentialWriterOption WRITER_OPTION = + SequentialWriterOption.newBuilder() + .trickleFsync(false) + .bufferSize(2 << 20) + .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 BigTableBlockWriter(Descriptor descriptor, TableMetadataRef metadata, LifecycleTransaction txn, final Set components) { - super(descriptor, ImmutableSet.copyOf(components), metadata, - DatabaseDescriptor.getDiskOptimizationStrategy()); + super(descriptor, ImmutableSet.copyOf(components), metadata, DatabaseDescriptor.getDiskOptimizationStrategy()); + txn.trackNew(this); this.metadata = metadata; - this.txn = txn; - this.componentWriters = new HashMap<>(components.size()); + this.componentWriters = new EnumMap<>(Component.Type.class); - assert supportedComponents.containsAll(components) : String.format("Unsupported streaming component detected %s", - new HashSet(components).removeAll(supportedComponents)); + if (!SUPPORTED_COMPONENTS.containsAll(components)) + throw new AssertionError(format("Unsupported streaming component detected %s", + Sets.difference(components, SUPPORTED_COMPONENTS))); for (Component c : components) - componentWriters.put(c.type, makeWriter(descriptor, c, writerOption)); + componentWriters.put(c.type, makeWriter(descriptor, c)); } - private static SequentialWriter makeWriter(Descriptor descriptor, Component component, SequentialWriterOption writerOption) + private static SequentialWriter makeWriter(Descriptor descriptor, Component component) { - return new SequentialWriter(new File(descriptor.filenameFor(component)), writerOption, false); + return new SequentialWriter(new File(descriptor.filenameFor(component)), WRITER_OPTION, false); } private void write(DataInputPlus in, long size, SequentialWriter out) throws FSWriteError { - final int BUFFER_SIZE = 1 * 1024 * 1024; + final int BUFFER_SIZE = 1 << 20; long bytesRead = 0; byte[] buff = new byte[BUFFER_SIZE]; try @@ -193,7 +202,8 @@ public void close() public void writeComponent(Component.Type type, DataInputPlus in, long size) { - logger.info("Writing component {} to {} length {}", type, componentWriters.get(type).getPath(), size); + logger.info("Writing component {} to {} length {}", type, componentWriters.get(type).getPath(), prettyPrintMemory(size)); + if (in instanceof RebufferingByteBufDataInputPlus) write((RebufferingByteBufDataInputPlus) in, size, componentWriters.get(type)); else @@ -202,18 +212,18 @@ public void writeComponent(Component.Type type, DataInputPlus in, long size) private void write(RebufferingByteBufDataInputPlus in, long size, SequentialWriter writer) { - logger.info("Block Writing component to {} length {}", writer.getPath(), size); + logger.info("Block Writing component to {} length {}", writer.getPath(), prettyPrintMemory(size)); try { long bytesWritten = in.consumeUntil(writer, size); if (bytesWritten != size) - throw new IOException(String.format("Failed to read correct number of bytes from Channel {}", writer)); + throw new IOException(format("Failed to read correct number of bytes from channel %s", writer)); } catch (IOException e) { throw new FSWriteError(e, writer.getPath()); } } -} +} \ No newline at end of file From 8e65f5c405d747c2958c51fdc25e42224a89bd06 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 25 Jul 2018 17:21:26 -0700 Subject: [PATCH 45/64] Address code review comments. Refactor ComponentManifest. --- .../streaming/CassandraBlockStreamReader.java | 13 ++- .../streaming/CassandraBlockStreamWriter.java | 51 +++++---- .../db/streaming/CassandraOutgoingFile.java | 4 +- .../db/streaming/CassandraStreamHeader.java | 2 +- .../db/streaming/ComponentManifest.java | 101 +++++++++--------- .../cassandra/io/sstable/Component.java | 18 +++- .../format/big/BigTableBlockWriter.java | 5 +- .../streaming/CassandraStreamHeaderTest.java | 3 - .../db/streaming/ComponentManifestTest.java | 7 +- .../cassandra/io/sstable/ComponentTest.java | 2 +- 10 files changed, 114 insertions(+), 92 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index d5295d49efdd..ef69d728b22a 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -20,7 +20,7 @@ import java.io.File; import java.io.IOException; -import java.util.Set; +import java.util.Collection; import com.google.common.base.Throwables; import org.slf4j.Logger; @@ -41,7 +41,6 @@ import org.apache.cassandra.streaming.messages.StreamMessageHeader; import static java.lang.String.format; - import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; /** @@ -88,7 +87,7 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException } ComponentManifest manifest = header.componentManifest; - long totalSize = manifest.getTotalSize(); + long totalSize = manifest.totalSize(); logger.debug("[Stream #{}] Started receiving sstable #{} from {}, size = {}, table = {}", session.planId(), @@ -101,11 +100,11 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException try { - writer = createWriter(cfs, totalSize, manifest.getComponents()); + writer = createWriter(cfs, totalSize, manifest.components()); long bytesRead = 0; - for (Component component : manifest.getComponents()) + for (Component component : manifest.components()) { - long length = manifest.getSizeForType(component.type); + long length = manifest.sizeOf(component); logger.debug("[Stream #{}] Started receiving {} component from {}, componentSize = {}, readBytes = {}, totalSize = {}", session.planId(), @@ -156,7 +155,7 @@ private File getDataDir(ColumnFamilyStore cfs, long totalSize) throws IOExceptio } @SuppressWarnings("resource") - protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize, Set components) throws IOException + protected BigTableBlockWriter createWriter(ColumnFamilyStore cfs, long totalSize, Collection components) throws IOException { File dataDir = getDataDir(cfs, totalSize); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index ffd92b4bd4f7..b3a9c894bfa7 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.nio.channels.FileChannel; -import java.util.List; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,9 +32,9 @@ import org.apache.cassandra.streaming.ProgressInfo; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamSession; -import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.streaming.StreamManager.StreamRateLimiter; +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; /** * CassandraBlockStreamWriter streams the entire SSTable to given channel. @@ -67,14 +66,18 @@ public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, @Override public void write(DataOutputStreamPlus output) throws IOException { - long totalSize = manifest.getTotalSize(); - logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}", session.planId(), - sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize); + long totalSize = manifest.totalSize(); + logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}", + session.planId(), + sstable.getFilename(), + session.peer, + sstable.getSSTableMetadata().repairedAt, + prettyPrintMemory(totalSize)); long progress = 0L; ByteBufDataOutputStreamPlus byteBufDataOutputStreamPlus = (ByteBufDataOutputStreamPlus) output; - for (Component component : manifest.getComponents()) + for (Component component : manifest.components()) { @SuppressWarnings("resource") // this is closed after the file is transferred by ByteBufDataOutputStreamPlus FileChannel in = new RandomAccessFile(sstable.descriptor.filenameFor(component), "r").getChannel(); @@ -83,27 +86,37 @@ public void write(DataOutputStreamPlus output) throws IOException long length = in.size(); // tracks write progress - long bytesRead = 0; logger.debug("[Stream #{}] Block streaming {}.{} gen {} component {} size {}", session.planId(), - sstable.getKeyspaceName(), sstable.getColumnFamilyName(), sstable.descriptor.generation, component, length); + sstable.getKeyspaceName(), + sstable.getColumnFamilyName(), + sstable.descriptor.generation, + component, length); - bytesRead += byteBufDataOutputStreamPlus.writeToChannel(in, limiter); + long bytesRead = byteBufDataOutputStreamPlus.writeToChannel(in, limiter); progress += bytesRead; - session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesRead, - length); + session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesRead, length); logger.debug("[Stream #{}] Finished block streaming {}.{} gen {} component {} to {}, xfered = {}, length = {}, totalSize = {}", - session.planId(), sstable.getKeyspaceName(), sstable.getColumnFamilyName(), - sstable.descriptor.generation, component, session.peer, FBUtilities.prettyPrintMemory(bytesRead), - FBUtilities.prettyPrintMemory(length), FBUtilities.prettyPrintMemory(totalSize)); + session.planId(), + sstable.getKeyspaceName(), + sstable.getColumnFamilyName(), + sstable.descriptor.generation, + component, + session.peer, + prettyPrintMemory(bytesRead), + prettyPrintMemory(length), + prettyPrintMemory(totalSize)); + } - byteBufDataOutputStreamPlus.flush(); + byteBufDataOutputStreamPlus.flush(); - logger.debug("[Stream #{}] Finished block streaming sstable {} to {}, xfered = {}, totalSize = {}", - session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), - FBUtilities.prettyPrintMemory(totalSize)); + logger.debug("[Stream #{}] Finished block streaming sstable {} to {}, xfered = {}, totalSize = {}", + session.planId(), + sstable.getFilename(), + session.peer, + prettyPrintMemory(progress), + prettyPrintMemory(totalSize)); - } } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index fb82daa40271..cc50c1a8f150 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -98,12 +98,12 @@ public CassandraOutgoingFile(StreamOperation operation, Ref ref, @VisibleForTesting public static ComponentManifest getComponentManifest(SSTableReader sstable) { - LinkedHashMap components = new LinkedHashMap<>(STREAM_COMPONENTS.size()); + LinkedHashMap components = new LinkedHashMap<>(STREAM_COMPONENTS.size()); for (Component component : STREAM_COMPONENTS) { File file = new File(sstable.descriptor.filenameFor(component)); if (file.exists()) - components.put(component.type, file.length()); + components.put(component, file.length()); } return new ComponentManifest(components); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index 6230aa722973..e16d79c20c1a 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -140,7 +140,7 @@ public long size() private long calculateSize() { if (fullStream) - return componentManifest.getTotalSize(); + return componentManifest.totalSize(); long transferSize = 0; if (compressionInfo != null) diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java index 27df8f1d5f3b..2991883457e2 100644 --- a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java +++ b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java @@ -19,12 +19,13 @@ package org.apache.cassandra.db.streaming; import java.io.IOException; -import java.util.Collections; +import java.util.ArrayList; +import java.util.Iterator; import java.util.LinkedHashMap; -import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.Set; + +import com.google.common.collect.Iterators; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.IVersionedSerializer; @@ -32,77 +33,78 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -public class ComponentManifest +public final class ComponentManifest implements Iterable { - private final LinkedHashMap manifest; - private final Set components = new LinkedHashSet<>(Component.Type.values().length); - private final long totalSize; + private final LinkedHashMap components; - public ComponentManifest(Map componentManifest) + public ComponentManifest(Map components) { - this.manifest = new LinkedHashMap<>(componentManifest); - - long size = 0; - for (Map.Entry entry : this.manifest.entrySet()) - { - size += entry.getValue(); - this.components.add(Component.parse(entry.getKey().repr)); - } - - this.totalSize = size; + this.components = new LinkedHashMap<>(components); } - public Long getSizeForType(Component.Type type) + public long sizeOf(Component component) { - return manifest.get(type); + Long size = components.get(component); + if (size == null) + throw new IllegalArgumentException("Component " + component + " is not present in the manifest"); + return size; } - public long getTotalSize() + public long totalSize() { + long totalSize = 0; + for (Long size : components.values()) + totalSize += size; return totalSize; } - public Set getComponents() + public List components() { - return Collections.unmodifiableSet(components); + return new ArrayList<>(components.keySet()); } + @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) + return true; + + if (!(o instanceof ComponentManifest)) + return false; + ComponentManifest that = (ComponentManifest) o; - return totalSize == that.totalSize && - Objects.equals(manifest, that.manifest); + return components.equals(that.components); } + @Override public int hashCode() { - - return Objects.hash(manifest, totalSize); + return components.hashCode(); } public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(ComponentManifest manifest, DataOutputPlus out, int version) throws IOException { - out.writeInt(manifest.manifest.size()); - for (Map.Entry entry : manifest.manifest.entrySet()) - serialize(entry.getKey(), entry.getValue(), out); + out.writeUnsignedVInt(manifest.components.size()); + for (Map.Entry entry : manifest.components.entrySet()) + { + out.writeByte(entry.getKey().type.id); + out.writeUnsignedVInt(entry.getValue()); + } } public ComponentManifest deserialize(DataInputPlus in, int version) throws IOException { - LinkedHashMap components = new LinkedHashMap<>(Component.Type.values().length); + int size = (int) in.readUnsignedVInt(); - int size = in.readInt(); - assert size >= 0 : "Invalid number of components"; + LinkedHashMap components = new LinkedHashMap<>(size); for (int i = 0; i < size; i++) { - Component.Type type = Component.Type.fromRepresentation(in.readByte()); - long length = in.readLong(); - components.put(type, length); + Component component = Component.get(Component.Type.fromRepresentation(in.readByte())); + long length = in.readUnsignedVInt(); + components.put(component, length); } return new ComponentManifest(components); @@ -110,20 +112,19 @@ public ComponentManifest deserialize(DataInputPlus in, int version) throws IOExc public long serializedSize(ComponentManifest manifest, int version) { - long size = 0; - size += TypeSizes.sizeof(manifest.manifest.size()); - for (Map.Entry entry : manifest.manifest.entrySet()) + long size = TypeSizes.sizeofUnsignedVInt(manifest.components.size()); + for (Map.Entry entry : manifest.components.entrySet()) { - size += TypeSizes.sizeof(entry.getKey().id); - size += TypeSizes.sizeof(entry.getValue()); + size += TypeSizes.sizeof(entry.getKey().type.id); + size += TypeSizes.sizeofUnsignedVInt(entry.getValue()); } return size; } - - private void serialize(Component.Type type, long size, DataOutputPlus out) throws IOException - { - out.writeByte(type.id); - out.writeLong(size); - } }; + + @Override + public Iterator iterator() + { + return Iterators.unmodifiableIterator(components.keySet().iterator()); + } } diff --git a/src/java/org/apache/cassandra/io/sstable/Component.java b/src/java/org/apache/cassandra/io/sstable/Component.java index a27fda1cd4c5..229a2bb3e9e6 100644 --- a/src/java/org/apache/cassandra/io/sstable/Component.java +++ b/src/java/org/apache/cassandra/io/sstable/Component.java @@ -145,7 +145,21 @@ public String name() public static Component parse(String name) { Type type = Type.fromRepresentation(name); + Component c = get(type); + if (c != null) + return c; + + switch (type) + { + case SECONDARY_INDEX: return new Component(Type.SECONDARY_INDEX, name); + case CUSTOM: return new Component(Type.CUSTOM, name); + default: throw new AssertionError(); + } + } + + public static Component get(Type type) + { // Build (or retrieve singleton for) the component object switch (type) { @@ -158,9 +172,7 @@ public static Component parse(String name) case CRC: return Component.CRC; case SUMMARY: return Component.SUMMARY; case TOC: return Component.TOC; - case SECONDARY_INDEX: return new Component(Type.SECONDARY_INDEX, name); - case CUSTOM: return new Component(Type.CUSTOM, name); - default: throw new AssertionError(); + default: return null; } } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java index 34f296f8a9c6..bd6b61b7992c 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java @@ -22,7 +22,6 @@ import java.util.Collection; import java.util.EnumMap; import java.util.Map; -import java.util.Set; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -78,7 +77,7 @@ public class BigTableBlockWriter extends SSTable implements SSTableMultiWriter public BigTableBlockWriter(Descriptor descriptor, TableMetadataRef metadata, LifecycleTransaction txn, - final Set components) + final Collection components) { super(descriptor, ImmutableSet.copyOf(components), metadata, DatabaseDescriptor.getDiskOptimizationStrategy()); @@ -88,7 +87,7 @@ public BigTableBlockWriter(Descriptor descriptor, if (!SUPPORTED_COMPONENTS.containsAll(components)) throw new AssertionError(format("Unsupported streaming component detected %s", - Sets.difference(components, SUPPORTED_COMPONENTS))); + Sets.difference(ImmutableSet.copyOf(components), SUPPORTED_COMPONENTS))); for (Component c : components) componentWriters.put(c.type, makeWriter(descriptor, c)); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index 02acc27997d2..b1207d89fbd5 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -21,10 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.junit.Test; diff --git a/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java index f5874fbdca38..f8d9df3880c7 100644 --- a/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.streaming; +import java.io.EOFException; import java.io.IOException; import java.util.HashMap; @@ -40,15 +41,15 @@ public class ComponentManifestTest @Test public void testSerialization() { - ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.Type.DATA, 100L); }}); + ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.DATA, 100L); }}); SerializationUtils.assertSerializationCycle(expected, ComponentManifest.serializer); } - @Test(expected = AssertionError.class) + @Test(expected = EOFException.class) public void testSerialization_FailsOnBadBytes() throws IOException { ByteBuf buf = Unpooled.buffer(512); - ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.Type.DATA, 100L); }}); + ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.DATA, 100L); }}); DataOutputPlus output = new ByteBufDataOutputPlus(buf); ComponentManifest.serializer.serialize(expected, output, MessagingService.VERSION_40); diff --git a/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java b/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java index a094d3043ace..21c58500bd35 100644 --- a/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java @@ -20,7 +20,7 @@ import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertTrue; public class ComponentTest { From 3f86fb7f690d8d8516d80f7c13c86cd7003c070a Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 25 Jul 2018 17:39:52 -0700 Subject: [PATCH 46/64] COF - Do not use zero copy path if legacy counter shards are present --- .../apache/cassandra/db/streaming/CassandraOutgoingFile.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index cc50c1a8f150..46981a2f2d27 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -181,7 +181,8 @@ public boolean shouldStreamFullSSTable() if (cfs == null) return false; - if (cfs.getCompactionStrategyManager().getCompactionStrategyFor(ref.get()) instanceof LeveledCompactionStrategy) + if (cfs.getCompactionStrategyManager().getCompactionStrategyFor(ref.get()) instanceof LeveledCompactionStrategy && + !ref.get().getSSTableMetadata().hasLegacyCounterShards) return isFullSSTableTransfersEnabled && isFullyContained; return false; From 78d9244ac14932e9d68ac1b83c1e00333b9a88b5 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 25 Jul 2018 17:45:13 -0700 Subject: [PATCH 47/64] Update documentation --- conf/cassandra.yaml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 60c10dc4bfc3..ab098eebe2d1 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -790,10 +790,9 @@ sstable_preemptive_open_interval_in_mb: 50 # Enabling this will reduce the GC pressure on sending and receiving node. # When unset, the default is enabled. While this feature tries to keep the # disks balanced, it cannot guarantee it. This feature will be automatically -# disabled if internode encryption is enabled. Compaction strategies such as -# LeveledCompaction will benefit the most while SizeTieredCompaction will -# see gains only in very limited cases. Most strategies will benefit if used -# in combination with CASSANDRA-6696 +# disabled if internode encryption is enabled. Currently this can be used with +# Leveled Compaction. Once CASSANDRA-14586 is fixed other compaction strategies +# will benefit as well when used in combination with CASSANDRA-6696. # streaming_zerocopy_sstables_enabled: false # Throttles all outbound streaming file transfers on this node to the From 7a01156e52dcde432fe91c499434a4ea7dafd3a2 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 25 Jul 2018 17:48:44 -0700 Subject: [PATCH 48/64] Adjust unit test --- .../cassandra/db/streaming/CassandraStreamHeaderTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index b1207d89fbd5..d5a5ffe77410 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -63,7 +63,7 @@ public void serializerTest_FullSSTableTransfer() String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); - ComponentManifest manifest = new ComponentManifest(new HashMap(ImmutableMap.of(Component.Type.DATA, 100L))); + ComponentManifest manifest = new ComponentManifest(new HashMap(ImmutableMap.of(Component.DATA, 100L))); CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, SSTableFormat.Type.BIG, From 5b6918860ed030cbcbddabdb18424f345b352c9e Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Wed, 25 Jul 2018 23:00:18 -0700 Subject: [PATCH 49/64] Rename variable to more accurate reflect its purpose --- .../db/streaming/CassandraBlockStreamWriter.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index b3a9c894bfa7..acc67ae2f8ce 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -92,10 +92,10 @@ public void write(DataOutputStreamPlus output) throws IOException sstable.descriptor.generation, component, length); - long bytesRead = byteBufDataOutputStreamPlus.writeToChannel(in, limiter); - progress += bytesRead; + long bytesWritten = byteBufDataOutputStreamPlus.writeToChannel(in, limiter); + progress += bytesWritten; - session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesRead, length); + session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesWritten, length); logger.debug("[Stream #{}] Finished block streaming {}.{} gen {} component {} to {}, xfered = {}, length = {}, totalSize = {}", session.planId(), @@ -104,7 +104,7 @@ public void write(DataOutputStreamPlus output) throws IOException sstable.descriptor.generation, component, session.peer, - prettyPrintMemory(bytesRead), + prettyPrintMemory(bytesWritten), prettyPrintMemory(length), prettyPrintMemory(totalSize)); } From 0eeb205f1fd1ddfb48aafd252c8878ad91a06ebc Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 00:29:06 -0700 Subject: [PATCH 50/64] COF - Refactor - lazy evaluate expensive containment and some stylistic changes --- .../db/streaming/CassandraOutgoingFile.java | 37 ++++++++++++++++--- .../streaming/CassandraOutgoingFileTest.java | 6 +-- 2 files changed, 34 insertions(+), 9 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 46981a2f2d27..445b0adebefc 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -33,7 +33,9 @@ import org.apache.cassandra.config.DatabaseDescriptor; 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.LeveledCompactionStrategy; +import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.Component; @@ -65,7 +67,7 @@ public class CassandraOutgoingFile implements OutgoingStream private final CassandraStreamHeader header; private final boolean keepSSTableLevel; private final ComponentManifest manifest; - private final boolean isFullyContained; + private Boolean isFullyContained; private final List> ranges; @@ -80,7 +82,6 @@ public CassandraOutgoingFile(StreamOperation operation, Ref ref, this.ranges = ImmutableList.copyOf(ranges); this.filename = ref.get().getFilename(); this.manifest = getComponentManifest(ref.get()); - this.isFullyContained = fullyContainedIn(this.ranges, ref.get()); SSTableReader sstable = ref.get(); keepSSTableLevel = operation == StreamOperation.BOOTSTRAP || operation == StreamOperation.REBUILD; @@ -176,20 +177,44 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) @VisibleForTesting public boolean shouldStreamFullSSTable() { + // don't stream if full sstable transfers are disabled or legacy counter shards are present + if (!isFullSSTableTransfersEnabled || ref.get().getSSTableMetadata().hasLegacyCounterShards) + return false; + ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(getTableId()); if (cfs == null) return false; - if (cfs.getCompactionStrategyManager().getCompactionStrategyFor(ref.get()) instanceof LeveledCompactionStrategy && - !ref.get().getSSTableMetadata().hasLegacyCounterShards) - return isFullSSTableTransfersEnabled && isFullyContained; + AbstractCompactionStrategy compactionStrategy = cfs.getCompactionStrategyManager() + .getCompactionStrategyFor(ref.get()); + + if (compactionStrategy instanceof LeveledCompactionStrategy) + return contained(ranges, ref.get()); + + if (compactionStrategy instanceof SizeTieredCompactionStrategy) + { + return (ranges != null + && ranges.size() == 1 + && ranges.get(0) + .contains(new Range<>(ref.get().first.getToken(), + ref.get().last.getToken()))); + } return false; } @VisibleForTesting - public boolean fullyContainedIn(List> normalizedRanges, SSTableReader sstable) + public boolean contained(List> normalizedRanges, SSTableReader sstable) + { + if (isFullyContained != null) + return isFullyContained; + + isFullyContained = computeContainment(normalizedRanges, sstable); + return isFullyContained; + } + + private boolean computeContainment(List> normalizedRanges, SSTableReader sstable) { if (normalizedRanges == null) return false; diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java index e1811380df2b..8256ac654870 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -93,7 +93,7 @@ public void validateFullyContainedIn_SingleContiguousRange_Succeeds() sstable.getPositionsForRanges(requestedRanges), requestedRanges, sstable.estimatedKeys()); - assertTrue(cof.fullyContainedIn(requestedRanges, sstable)); + assertTrue(cof.contained(requestedRanges, sstable)); } @Test @@ -105,7 +105,7 @@ public void validateFullyContainedIn_PartialOverlap_Fails() sstable.getPositionsForRanges(requestedRanges), requestedRanges, sstable.estimatedKeys()); - assertFalse(cof.fullyContainedIn(requestedRanges, sstable)); + assertFalse(cof.contained(requestedRanges, sstable)); } @Test @@ -119,7 +119,7 @@ public void validateFullyContainedIn_SplitRange_Succeeds() sstable.getPositionsForRanges(requestedRanges), requestedRanges, sstable.estimatedKeys()); - assertTrue(cof.fullyContainedIn(requestedRanges, sstable)); + assertTrue(cof.contained(requestedRanges, sstable)); } private DecoratedKey getKeyAtIndex(int i) From 2be8a8feb06d2ccb887ac41d99515b3ef892b333 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 09:23:32 -0700 Subject: [PATCH 51/64] add pretty printing to missing field. fix format specifier. formatting --- .../cassandra/db/streaming/CassandraBlockStreamWriter.java | 3 ++- .../apache/cassandra/db/streaming/CassandraStreamHeader.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index acc67ae2f8ce..9bb37c2a854e 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -90,7 +90,8 @@ public void write(DataOutputStreamPlus output) throws IOException sstable.getKeyspaceName(), sstable.getColumnFamilyName(), sstable.descriptor.generation, - component, length); + component, + prettyPrintMemory(length)); long bytesWritten = byteBufDataOutputStreamPlus.writeToChannel(in, limiter); progress += bytesWritten; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index e16d79c20c1a..10fde1851e5f 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -275,7 +275,7 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function ByteBuffer keyBuf = ByteBufferUtil.readWithVIntLength(in); IPartitioner partitioner = partitionerMapper.apply(tableId); if (partitioner == null) - throw new IllegalArgumentException(String.format("Could not determine partitioner for tableId {}", tableId)); + throw new IllegalArgumentException(String.format("Could not determine partitioner for tableId %s", tableId)); firstKey = partitioner.decorateKey(keyBuf); } From 5cf03f8ff15ecf227129a9bc279ec25c4731ceeb Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Thu, 26 Jul 2018 14:03:43 +0100 Subject: [PATCH 52/64] Introduce a Builder to CassandraStreamHeader also: - better naming - fix a bug in CassandraStreamHeader.serializedSize() - remove objects without a toString() implementation from CSH.toString() --- .../db/streaming/CassandraIncomingFile.java | 4 +- .../db/streaming/CassandraOutgoingFile.java | 27 ++- .../db/streaming/CassandraStreamHeader.java | 225 ++++++++++++------ .../db/streaming/CassandraStreamReader.java | 2 +- .../ZeroCopyStreamingBenchmark.java | 38 ++- .../CassandraBlockStreamWriterTest.java | 16 +- .../streaming/CassandraStreamHeaderTest.java | 43 ++-- 7 files changed, 236 insertions(+), 119 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java index be8504ff2507..3c5299415688 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java @@ -60,10 +60,10 @@ public CassandraIncomingFile(ColumnFamilyStore cfs, StreamSession session, Strea public synchronized void read(DataInputPlus in, int version) throws IOException { CassandraStreamHeader streamHeader = CassandraStreamHeader.serializer.deserialize(in, version); - logger.debug("Incoming stream fullStream={} components={}", streamHeader.fullStream, streamHeader.componentManifest); + logger.debug("Incoming stream entireSSTable={} components={}", streamHeader.isEntireSSTable, streamHeader.componentManifest); IStreamReader reader; - if (streamHeader.fullStream) + if (streamHeader.isEntireSSTable) { reader = new CassandraBlockStreamReader(header, streamHeader, session); } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 445b0adebefc..d826a4114e96 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -85,15 +85,20 @@ public CassandraOutgoingFile(StreamOperation operation, Ref ref, SSTableReader sstable = ref.get(); keepSSTableLevel = operation == StreamOperation.BOOTSTRAP || operation == StreamOperation.REBUILD; - this.header = new CassandraStreamHeader(sstable.descriptor.version, - sstable.descriptor.formatType, - estimatedKeys, - sections, - sstable.compression ? sstable.getCompressionMetadata() : null, - keepSSTableLevel ? sstable.getSSTableLevel() : 0, - sstable.header.toComponent(), manifest, shouldStreamFullSSTable(), - sstable.first, - sstable.metadata().id); + this.header = + CassandraStreamHeader.builder() + .withSSTableFormat(sstable.descriptor.formatType) + .withSSTableVersion(sstable.descriptor.version) + .withSSTableLevel(keepSSTableLevel ? sstable.getSSTableLevel() : 0) + .withEstimatedKeys(estimatedKeys) + .withSections(sections) + .withCompressionMetadata(sstable.compression ? sstable.getCompressionMetadata() : null) + .withSerializationHeader(sstable.header.toComponent()) + .isEntireSSTable(shouldStreamEntireSSTable()) + .withComponentManifest(manifest) + .withFirstKey(sstable.first) + .withTableId(sstable.metadata().id) + .build(); } @VisibleForTesting @@ -160,7 +165,7 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) out.flush(); IStreamWriter writer; - if (shouldStreamFullSSTable()) + if (shouldStreamEntireSSTable()) { writer = new CassandraBlockStreamWriter(sstable, session, manifest); } @@ -175,7 +180,7 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) } @VisibleForTesting - public boolean shouldStreamFullSSTable() + public boolean shouldStreamEntireSSTable() { // don't stream if full sstable transfers are disabled or legacy counter shards are present if (!isFullSSTableTransfersEnabled || ref.get().getSSTableMetadata().hasLegacyCounterShards) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java index 10fde1851e5f..2af56de9a0a8 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamHeader.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.db.streaming; import java.io.IOException; @@ -42,6 +41,8 @@ import org.apache.cassandra.schema.TableId; import org.apache.cassandra.utils.ByteBufferUtil; +import static com.google.common.base.Preconditions.checkNotNull; + public class CassandraStreamHeader { /** SSTable version */ @@ -59,10 +60,10 @@ public class CassandraStreamHeader private final CompressionMetadata compressionMetadata; public volatile CompressionInfo compressionInfo; public final int sstableLevel; - public final SerializationHeader.Component header; + public final SerializationHeader.Component serializationHeader; - /* flag indicating whether this is a partial or full sstable transfer */ - public final boolean fullStream; + /* flag indicating whether this is a partial or entire sstable transfer */ + public final boolean isEntireSSTable; /* first token of the sstable required for faster streaming */ public final DecoratedKey firstKey; public final TableId tableId; @@ -71,57 +72,26 @@ public class CassandraStreamHeader /* cached size value */ private transient final long size; - public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, - List sections, CompressionMetadata compressionMetadata, - CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header, - ComponentManifest componentManifest, - boolean fullStream, DecoratedKey firstKey, TableId tableId) - { - this.version = version; - this.format = format; - this.estimatedKeys = estimatedKeys; - this.sections = sections; - this.compressionMetadata = compressionMetadata; - this.compressionInfo = compressionInfo; - this.sstableLevel = sstableLevel; - this.header = header; - this.fullStream = fullStream; - this.componentManifest = componentManifest; - this.firstKey = firstKey; - this.tableId = tableId; - this.size = calculateSize(); - } - - private CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, - List sections, CompressionMetadata compressionMetadata, - CompressionInfo compressionInfo, int sstableLevel, SerializationHeader.Component header, TableId tableId) - { - this(version, format, estimatedKeys, sections, compressionMetadata, compressionInfo, sstableLevel, header, null, false, null, tableId); - } - - public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, - List sections, CompressionMetadata compressionMetadata, - int sstableLevel, SerializationHeader.Component header, TableId tableId) + private CassandraStreamHeader(Builder builder) { - this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header, tableId); + version = builder.version; + format = builder.format; + estimatedKeys = builder.estimatedKeys; + sections = builder.sections; + compressionMetadata = builder.compressionMetadata; + compressionInfo = builder.compressionInfo; + sstableLevel = builder.sstableLevel; + serializationHeader = builder.serializationHeader; + tableId = builder.tableId; + isEntireSSTable = builder.isEntireSSTable; + componentManifest = builder.componentManifest; + firstKey = builder.firstKey; + size = calculateSize(); } - public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, - List sections, CompressionMetadata compressionMetadata, - int sstableLevel, SerializationHeader.Component header, ComponentManifest componentManifest, - boolean fullStream, DecoratedKey firstKey, TableId tableId) + public static Builder builder() { - this(version, format, estimatedKeys, sections, compressionMetadata, null, sstableLevel, header, - componentManifest, fullStream, firstKey, tableId); - } - - public CassandraStreamHeader(Version version, SSTableFormat.Type format, long estimatedKeys, - List sections, CompressionInfo compressionInfo, - int sstableLevel, SerializationHeader.Component header, ComponentManifest componentManifest, - boolean fullStream, DecoratedKey firstKey, TableId tableId) - { - this(version, format, estimatedKeys, sections, null, compressionInfo, sstableLevel, header, - componentManifest, fullStream, firstKey, tableId); + return new Builder(); } public boolean isCompressed() @@ -139,7 +109,7 @@ public long size() private long calculateSize() { - if (fullStream) + if (isEntireSSTable) return componentManifest.totalSize(); long transferSize = 0; @@ -171,13 +141,11 @@ public String toString() ", format=" + format + ", estimatedKeys=" + estimatedKeys + ", sections=" + sections + - ", compressionInfo=" + compressionInfo + ", sstableLevel=" + sstableLevel + - ", header=" + header + - ", fullStream=" + fullStream + + ", header=" + serializationHeader + + ", isEntireSSTable=" + isEntireSSTable + ", firstKey=" + firstKey + ", tableId=" + tableId + - ", componentManifest=" + componentManifest + '}'; } @@ -188,12 +156,12 @@ public boolean equals(Object o) CassandraStreamHeader that = (CassandraStreamHeader) o; return estimatedKeys == that.estimatedKeys && sstableLevel == that.sstableLevel && - fullStream == that.fullStream && + isEntireSSTable == that.isEntireSSTable && Objects.equals(version, that.version) && format == that.format && Objects.equals(sections, that.sections) && Objects.equals(compressionInfo, that.compressionInfo) && - Objects.equals(header, that.header) && + Objects.equals(serializationHeader, that.serializationHeader) && Objects.equals(componentManifest, that.componentManifest) && Objects.equals(firstKey, that.firstKey) && Objects.equals(tableId, that.tableId); @@ -201,8 +169,8 @@ public boolean equals(Object o) public int hashCode() { - return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, header, componentManifest, - fullStream, firstKey, tableId); + return Objects.hash(version, format, estimatedKeys, sections, compressionInfo, sstableLevel, serializationHeader, componentManifest, + isEntireSSTable, firstKey, tableId); } public static final IVersionedSerializer serializer = new CassandraStreamHeaderSerializer(); @@ -225,12 +193,12 @@ public void serialize(CassandraStreamHeader header, DataOutputPlus out, int vers CompressionInfo.serializer.serialize(header.compressionInfo, out, version); out.writeInt(header.sstableLevel); - SerializationHeader.serializer.serialize(header.version, header.header, out); + SerializationHeader.serializer.serialize(header.version, header.serializationHeader, out); header.tableId.serialize(out); - out.writeBoolean(header.fullStream); + out.writeBoolean(header.isEntireSSTable); - if (header.fullStream) + if (header.isEntireSSTable) { ComponentManifest.serializer.serialize(header.componentManifest, out, version); ByteBufferUtil.writeWithVIntLength(header.firstKey.getKey(), out); @@ -265,11 +233,11 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function SerializationHeader.Component header = SerializationHeader.serializer.deserialize(sstableVersion, in); TableId tableId = TableId.deserialize(in); - boolean fullStream = in.readBoolean(); + boolean isEntireSSTable = in.readBoolean(); ComponentManifest manifest = null; DecoratedKey firstKey = null; - if (fullStream) + if (isEntireSSTable) { manifest = ComponentManifest.serializer.deserialize(in, version); ByteBuffer keyBuf = ByteBufferUtil.readWithVIntLength(in); @@ -279,8 +247,18 @@ public CassandraStreamHeader deserialize(DataInputPlus in, int version, Function firstKey = partitioner.decorateKey(keyBuf); } - return new CassandraStreamHeader(sstableVersion, format, estimatedKeys, sections, compressionInfo, - sstableLevel, header, manifest, fullStream, firstKey, tableId); + return builder().withSSTableFormat(format) + .withSSTableVersion(sstableVersion) + .withSSTableLevel(sstableLevel) + .withEstimatedKeys(estimatedKeys) + .withSections(sections) + .withCompressionInfo(compressionInfo) + .withSerializationHeader(header) + .withComponentManifest(manifest) + .isEntireSSTable(isEntireSSTable) + .withFirstKey(firstKey) + .withTableId(tableId) + .build(); } public long serializedSize(CassandraStreamHeader header, int version) @@ -296,15 +274,17 @@ public long serializedSize(CassandraStreamHeader header, int version) size += TypeSizes.sizeof(section.lowerPosition); size += TypeSizes.sizeof(section.upperPosition); } + + header.calculateCompressionInfo(); size += CompressionInfo.serializer.serializedSize(header.compressionInfo, version); size += TypeSizes.sizeof(header.sstableLevel); - size += SerializationHeader.serializer.serializedSize(header.version, header.header); + size += SerializationHeader.serializer.serializedSize(header.version, header.serializationHeader); size += header.tableId.serializedSize(); - size += TypeSizes.sizeof(header.fullStream); + size += TypeSizes.sizeof(header.isEntireSSTable); - if (header.fullStream) + if (header.isEntireSSTable) { size += ComponentManifest.serializer.serializedSize(header.componentManifest, version); size += ByteBufferUtil.serializedSizeWithVIntLength(header.firstKey.getKey()); @@ -312,4 +292,109 @@ public long serializedSize(CassandraStreamHeader header, int version) return size; } } + + public static final class Builder + { + private Version version; + private SSTableFormat.Type format; + private long estimatedKeys; + private List sections; + private CompressionMetadata compressionMetadata; + private CompressionInfo compressionInfo; + private int sstableLevel; + private SerializationHeader.Component serializationHeader; + private ComponentManifest componentManifest; + private boolean isEntireSSTable; + private DecoratedKey firstKey; + private TableId tableId; + + public Builder withSSTableFormat(SSTableFormat.Type format) + { + this.format = format; + return this; + } + + public Builder withSSTableVersion(Version version) + { + this.version = version; + return this; + } + + public Builder withSSTableLevel(int sstableLevel) + { + this.sstableLevel = sstableLevel; + return this; + } + + public Builder withEstimatedKeys(long estimatedKeys) + { + this.estimatedKeys = estimatedKeys; + return this; + } + + public Builder withSections(List sections) + { + this.sections = sections; + return this; + } + + public Builder withCompressionMetadata(CompressionMetadata compressionMetadata) + { + this.compressionMetadata = compressionMetadata; + return this; + } + + public Builder withCompressionInfo(CompressionInfo compressionInfo) + { + this.compressionInfo = compressionInfo; + return this; + } + + public Builder withSerializationHeader(SerializationHeader.Component header) + { + this.serializationHeader = header; + return this; + } + + public Builder withTableId(TableId tableId) + { + this.tableId = tableId; + return this; + } + + public Builder isEntireSSTable(boolean isEntireSSTable) + { + this.isEntireSSTable = isEntireSSTable; + return this; + } + + public Builder withComponentManifest(ComponentManifest componentManifest) + { + this.componentManifest = componentManifest; + return this; + } + + public Builder withFirstKey(DecoratedKey firstKey) + { + this.firstKey = firstKey; + return this; + } + + public CassandraStreamHeader build() + { + checkNotNull(version); + checkNotNull(format); + checkNotNull(sections); + checkNotNull(serializationHeader); + checkNotNull(tableId); + + if (isEntireSSTable) + { + checkNotNull(componentManifest); + checkNotNull(firstKey); + } + + return new CassandraStreamHeader(this); + } + } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java index f0d9c4ee6422..fccabfe08c6b 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java @@ -85,7 +85,7 @@ public CassandraStreamReader(StreamMessageHeader header, CassandraStreamHeader s this.pendingRepair = header.pendingRepair; this.format = streamHeader.format; this.sstableLevel = streamHeader.sstableLevel; - this.header = streamHeader.header; + this.header = streamHeader.serializationHeader; this.fileSeqNum = header.sequenceNumber; } diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index 6f627d4b2455..b61bd1ef356a 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -46,7 +46,6 @@ import org.apache.cassandra.db.streaming.CassandraStreamHeader; import org.apache.cassandra.db.streaming.CassandraStreamReader; import org.apache.cassandra.db.streaming.CassandraStreamWriter; -import org.apache.cassandra.db.streaming.CompressionInfo; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.SSTableMultiWriter; @@ -130,16 +129,24 @@ public void setupBenchmark() throws IOException session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, serializedBlockStream.readableBytes())); - CassandraStreamHeader blockStreamHeader = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, - sstable.estimatedKeys(), Collections.emptyList(), - (CompressionInfo) null, 0, sstable.header.toComponent(), - CassandraOutgoingFile.getComponentManifest(sstable), true, sstable.first, - sstable.metadata().id); + CassandraStreamHeader entireSSTableStreamHeader = + CassandraStreamHeader.builder() + .withSSTableFormat(sstable.descriptor.formatType) + .withSSTableVersion(sstable.descriptor.version) + .withSSTableLevel(0) + .withEstimatedKeys(sstable.estimatedKeys()) + .withSections(Collections.emptyList()) + .withSerializationHeader(sstable.header.toComponent()) + .withComponentManifest(CassandraOutgoingFile.getComponentManifest(sstable)) + .isEntireSSTable(true) + .withFirstKey(sstable.first) + .withTableId(sstable.metadata().id) + .build(); blockStreamReader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, - null), blockStreamHeader, session); + null), entireSSTableStreamHeader, session); List> requestedRanges = Arrays.asList(new Range<>(sstable.first.minValue().getToken(), sstable.last.getToken())); partialStreamWriter = new CassandraStreamWriter(sstable, sstable.getPositionsForRanges(requestedRanges), session); @@ -148,17 +155,22 @@ public void setupBenchmark() throws IOException partialStreamWriter.write(ByteBufDataOutputStreamPlus.create(session, partialStreamChannel, 1024 * 1024)); serializedPartialStream = partialStreamChannel.getSerializedStream(); - CassandraStreamHeader partialStreamHeader = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, - sstable.estimatedKeys(), sstable.getPositionsForRanges(requestedRanges), - (CompressionInfo) null, 0, sstable.header.toComponent(), - CassandraOutgoingFile.getComponentManifest(sstable), false, sstable.first, - sstable.metadata().id); + CassandraStreamHeader partialSSTableStreamHeader = + CassandraStreamHeader.builder() + .withSSTableFormat(sstable.descriptor.formatType) + .withSSTableVersion(sstable.descriptor.version) + .withSSTableLevel(0) + .withEstimatedKeys(sstable.estimatedKeys()) + .withSections(sstable.getPositionsForRanges(requestedRanges)) + .withSerializationHeader(sstable.header.toComponent()) + .withTableId(sstable.metadata().id) + .build(); partialStreamReader = new CassandraStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), - partialStreamHeader, session); + partialSSTableStreamHeader, session); } private Keyspace setupSchemaAndKeySpace() diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java index 8cd4dc3d2ef2..e139dd344844 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java @@ -15,10 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.db.streaming; - import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; @@ -141,7 +139,19 @@ public void testBlockReadingAndWritingOverWire() throws Exception session.prepareReceiving(new StreamSummary(sstable.metadata().id, 1, 5104)); - CassandraStreamHeader header = new CassandraStreamHeader(sstable.descriptor.version, sstable.descriptor.formatType, sstable.estimatedKeys(), Collections.emptyList(), (CompressionInfo) null, 0, sstable.header.toComponent(), CassandraOutgoingFile.getComponentManifest(sstable), true, sstable.first, sstable.metadata().id); + CassandraStreamHeader header = + CassandraStreamHeader.builder() + .withSSTableFormat(sstable.descriptor.formatType) + .withSSTableVersion(sstable.descriptor.version) + .withSSTableLevel(0) + .withEstimatedKeys(sstable.estimatedKeys()) + .withSections(Collections.emptyList()) + .withSerializationHeader(sstable.header.toComponent()) + .withComponentManifest(CassandraOutgoingFile.getComponentManifest(sstable)) + .isEntireSSTable(true) + .withFirstKey(sstable.first) + .withTableId(sstable.metadata().id) + .build(); CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index d5a5ffe77410..07a856f8327f 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -15,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.db.streaming; import java.io.IOException; -import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import com.google.common.collect.ImmutableMap; @@ -29,7 +28,6 @@ import org.apache.cassandra.db.SerializationHeader; import org.apache.cassandra.db.streaming.CassandraStreamHeader.CassandraStreamHeaderSerializer; import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableFormat; import org.apache.cassandra.io.sstable.format.big.BigFormat; @@ -45,14 +43,16 @@ public void serializerTest() { String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); - CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, - SSTableFormat.Type.BIG, - 0, - new ArrayList<>(), - ((CompressionMetadata) null), - 0, - SerializationHeader.makeWithoutStats(metadata).toComponent(), - metadata.id); + CassandraStreamHeader header = + CassandraStreamHeader.builder() + .withSSTableFormat(SSTableFormat.Type.BIG) + .withSSTableVersion(BigFormat.latestVersion) + .withSSTableLevel(0) + .withEstimatedKeys(0) + .withSections(Collections.emptyList()) + .withSerializationHeader(SerializationHeader.makeWithoutStats(metadata).toComponent()) + .withTableId(metadata.id) + .build(); SerializationUtils.assertSerializationCycle(header, CassandraStreamHeader.serializer); } @@ -65,14 +65,19 @@ public void serializerTest_FullSSTableTransfer() ComponentManifest manifest = new ComponentManifest(new HashMap(ImmutableMap.of(Component.DATA, 100L))); - CassandraStreamHeader header = new CassandraStreamHeader(BigFormat.latestVersion, - SSTableFormat.Type.BIG, - 0, - new ArrayList<>(), - ((CompressionMetadata) null), - 0, - SerializationHeader.makeWithoutStats(metadata).toComponent(), - manifest, true, Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER), metadata.id); + CassandraStreamHeader header = + CassandraStreamHeader.builder() + .withSSTableFormat(SSTableFormat.Type.BIG) + .withSSTableVersion(BigFormat.latestVersion) + .withSSTableLevel(0) + .withEstimatedKeys(0) + .withSections(Collections.emptyList()) + .withSerializationHeader(SerializationHeader.makeWithoutStats(metadata).toComponent()) + .withComponentManifest(manifest) + .isEntireSSTable(true) + .withFirstKey(Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER)) + .withTableId(metadata.id) + .build(); SerializationUtils.assertSerializationCycle(header, new TestableCassandraStreamHeaderSerializer()); } From 128faf9717fc20abe1445047c316c0f52904e6da Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 10:01:06 -0700 Subject: [PATCH 53/64] Make naming consistent --- .../org/apache/cassandra/config/DatabaseDescriptor.java | 2 +- .../cassandra/db/streaming/CassandraOutgoingFile.java | 4 ++-- .../cassandra/db/streaming/CassandraStreamHeaderTest.java | 7 +++---- .../cassandra/db/streaming/ComponentManifestTest.java | 6 +++--- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index e2d14d0a101a..affbc94c3116 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2260,7 +2260,7 @@ public static int getStreamingConnectionsPerHost() return conf.streaming_connections_per_host; } - public static boolean isFullSSTableTransfersEnabled() + public static boolean isZeroCopySSTableTransfersEnabled() { if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) { diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index d826a4114e96..dd41cc4e9149 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -55,7 +55,7 @@ */ public class CassandraOutgoingFile implements OutgoingStream { - private static final boolean isFullSSTableTransfersEnabled = DatabaseDescriptor.isFullSSTableTransfersEnabled(); + private static final boolean isZeroCopySSTableTransfersEnabled = DatabaseDescriptor.isZeroCopySSTableTransfersEnabled(); public 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); @@ -183,7 +183,7 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) public boolean shouldStreamEntireSSTable() { // don't stream if full sstable transfers are disabled or legacy counter shards are present - if (!isFullSSTableTransfersEnabled || ref.get().getSSTableMetadata().hasLegacyCounterShards) + if (!isZeroCopySSTableTransfersEnabled || ref.get().getSSTableMetadata().hasLegacyCounterShards) return false; ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(getTableId()); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java index 07a856f8327f..809743b56122 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java @@ -19,9 +19,8 @@ import java.io.IOException; import java.util.Collections; -import java.util.HashMap; +import java.util.LinkedHashMap; -import com.google.common.collect.ImmutableMap; import org.junit.Test; import org.apache.cassandra.cql3.statements.CreateTableStatement; @@ -58,12 +57,12 @@ public void serializerTest() } @Test - public void serializerTest_FullSSTableTransfer() + public void serializerTest_EntireSSTableTransfer() { String ddl = "CREATE TABLE tbl (k INT PRIMARY KEY, v INT)"; TableMetadata metadata = CreateTableStatement.parse(ddl, "ks").build(); - ComponentManifest manifest = new ComponentManifest(new HashMap(ImmutableMap.of(Component.DATA, 100L))); + ComponentManifest manifest = new ComponentManifest(new LinkedHashMap() {{ put(Component.DATA, 100L); }}); CassandraStreamHeader header = CassandraStreamHeader.builder() diff --git a/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java index f8d9df3880c7..f478a008d908 100644 --- a/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java @@ -20,7 +20,7 @@ import java.io.EOFException; import java.io.IOException; -import java.util.HashMap; +import java.util.LinkedHashMap; import org.junit.Test; @@ -41,7 +41,7 @@ public class ComponentManifestTest @Test public void testSerialization() { - ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.DATA, 100L); }}); + ComponentManifest expected = new ComponentManifest(new LinkedHashMap() {{ put(Component.DATA, 100L); }}); SerializationUtils.assertSerializationCycle(expected, ComponentManifest.serializer); } @@ -49,7 +49,7 @@ public void testSerialization() public void testSerialization_FailsOnBadBytes() throws IOException { ByteBuf buf = Unpooled.buffer(512); - ComponentManifest expected = new ComponentManifest(new HashMap() {{ put(Component.DATA, 100L); }}); + ComponentManifest expected = new ComponentManifest(new LinkedHashMap() {{ put(Component.DATA, 100L); }}); DataOutputPlus output = new ByteBufDataOutputPlus(buf); ComponentManifest.serializer.serialize(expected, output, MessagingService.VERSION_40); From 2ba41c15a708f68ff30bfc1b4d229ea753a5c5a2 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 14:37:43 -0700 Subject: [PATCH 54/64] Make sure we disable zero copy and warn once. --- .../cassandra/config/DatabaseDescriptor.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index affbc94c3116..a06f7ad1d2dd 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -708,6 +708,15 @@ else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_i "server_encryption_options.internode_encryption = " + conf.server_encryption_options.internode_encryption, false); } + if (conf.streaming_zerocopy_sstables_enabled) + { + if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) + { + logger.warn("Internode encryption enabled. Disabling zero copy SSTable transfers for streaming."); + conf.streaming_zerocopy_sstables_enabled = false; + } + } + if (conf.max_value_size_in_mb <= 0) throw new ConfigurationException("max_value_size_in_mb must be positive", false); else if (conf.max_value_size_in_mb >= 2048) @@ -2262,16 +2271,7 @@ public static int getStreamingConnectionsPerHost() public static boolean isZeroCopySSTableTransfersEnabled() { - if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) - { - logger.warn("Internode encryption enabled. Disabling zero copy SSTable transfers for streaming."); - return false; - } - else - { - return Boolean.valueOf(System.getProperty(Config.PROPERTY_PREFIX + "streaming.enableZeroCopySSTableTransfers", - Boolean.toString(conf.streaming_zerocopy_sstables_enabled))); - } + return conf.streaming_zerocopy_sstables_enabled; } public static String getLocalDataCenter() From 1898747f6a9ef35182bdc7101523913019ea7191 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 14:41:52 -0700 Subject: [PATCH 55/64] Stylistic changes --- conf/cassandra.yaml | 2 +- src/java/org/apache/cassandra/config/Config.java | 2 +- .../org/apache/cassandra/config/DatabaseDescriptor.java | 6 +++--- test/conf/cassandra.yaml | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index ab098eebe2d1..58ab3d1bdb81 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -793,7 +793,7 @@ sstable_preemptive_open_interval_in_mb: 50 # disabled if internode encryption is enabled. Currently this can be used with # Leveled Compaction. Once CASSANDRA-14586 is fixed other compaction strategies # will benefit as well when used in combination with CASSANDRA-6696. -# streaming_zerocopy_sstables_enabled: false +# streaming_zero_copy_sstables_enabled: false # Throttles all outbound streaming file transfers on this node to the # given total throughput in Mbps. This is necessary because Cassandra does diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index ba9214fbf834..1831048258f7 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -380,7 +380,7 @@ public class Config public int block_for_peers_timeout_in_secs = 10; public volatile boolean automatic_sstable_upgrade = false; public volatile int max_concurrent_automatic_sstable_upgrades = 1; - public boolean streaming_zerocopy_sstables_enabled = true; + public boolean streaming_zero_copy_sstables_enabled = true; public volatile AuditLogOptions audit_logging_options = new AuditLogOptions(); diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index a06f7ad1d2dd..820f1fce19fc 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -708,12 +708,12 @@ else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_i "server_encryption_options.internode_encryption = " + conf.server_encryption_options.internode_encryption, false); } - if (conf.streaming_zerocopy_sstables_enabled) + if (conf.streaming_zero_copy_sstables_enabled) { if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) { logger.warn("Internode encryption enabled. Disabling zero copy SSTable transfers for streaming."); - conf.streaming_zerocopy_sstables_enabled = false; + conf.streaming_zero_copy_sstables_enabled = false; } } @@ -2271,7 +2271,7 @@ public static int getStreamingConnectionsPerHost() public static boolean isZeroCopySSTableTransfersEnabled() { - return conf.streaming_zerocopy_sstables_enabled; + return conf.streaming_zero_copy_sstables_enabled; } public static String getLocalDataCenter() diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml index 29baf5627eec..71f8c6e7fc8f 100644 --- a/test/conf/cassandra.yaml +++ b/test/conf/cassandra.yaml @@ -46,5 +46,5 @@ enable_user_defined_functions: true enable_scripted_user_defined_functions: true prepared_statements_cache_size_mb: 1 corrupted_tombstone_strategy: exception -streaming_zerocopy_sstables_enabled: true +streaming_zero_copy_sstables_enabled: true stream_throughput_outbound_megabits_per_sec: 200000000 From 230bf3b4421e9d383d755547f0cc41bba7ce15d4 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 17:00:06 -0700 Subject: [PATCH 56/64] Remove IStreamWriter --- .../streaming/CassandraBlockStreamWriter.java | 12 +++---- .../db/streaming/CassandraOutgoingFile.java | 11 ++++--- .../db/streaming/CassandraStreamWriter.java | 3 +- .../cassandra/db/streaming/IStreamWriter.java | 31 ------------------- 4 files changed, 12 insertions(+), 45 deletions(-) delete mode 100644 src/java/org/apache/cassandra/db/streaming/IStreamWriter.java diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index 9bb37c2a854e..fdf1a422cea2 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -39,7 +39,7 @@ /** * CassandraBlockStreamWriter streams the entire SSTable to given channel. */ -public class CassandraBlockStreamWriter implements IStreamWriter +public class CassandraBlockStreamWriter { private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamWriter.class); @@ -60,11 +60,10 @@ public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, * Stream the entire file to given channel. *

* - * @param output where this writes data to + * @param out where this writes data to * @throws IOException on any I/O error */ - @Override - public void write(DataOutputStreamPlus output) throws IOException + public void write(ByteBufDataOutputStreamPlus out) throws IOException { long totalSize = manifest.totalSize(); logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}", @@ -75,7 +74,6 @@ public void write(DataOutputStreamPlus output) throws IOException prettyPrintMemory(totalSize)); long progress = 0L; - ByteBufDataOutputStreamPlus byteBufDataOutputStreamPlus = (ByteBufDataOutputStreamPlus) output; for (Component component : manifest.components()) { @@ -93,7 +91,7 @@ public void write(DataOutputStreamPlus output) throws IOException component, prettyPrintMemory(length)); - long bytesWritten = byteBufDataOutputStreamPlus.writeToChannel(in, limiter); + long bytesWritten = out.writeToChannel(in, limiter); progress += bytesWritten; session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesWritten, length); @@ -110,7 +108,7 @@ public void write(DataOutputStreamPlus output) throws IOException prettyPrintMemory(totalSize)); } - byteBufDataOutputStreamPlus.flush(); + out.flush(); logger.debug("[Stream #{}] Finished block streaming sstable {} to {}, xfered = {}, totalSize = {}", session.planId(), diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index dd41cc4e9149..782e50297237 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -42,6 +42,7 @@ import org.apache.cassandra.io.sstable.KeyIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataOutputStreamPlus; +import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.streaming.OutgoingStream; import org.apache.cassandra.streaming.StreamOperation; @@ -164,19 +165,19 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) CassandraStreamHeader.serializer.serialize(header, out, version); out.flush(); - IStreamWriter writer; - if (shouldStreamEntireSSTable()) + if (shouldStreamEntireSSTable() && out instanceof ByteBufDataOutputStreamPlus) { - writer = new CassandraBlockStreamWriter(sstable, session, manifest); + CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, manifest); + writer.write((ByteBufDataOutputStreamPlus) out); } else { - writer = (header.compressionInfo == null) ? + CassandraStreamWriter writer = (header.compressionInfo == null) ? new CassandraStreamWriter(sstable, header.sections, session) : new CompressedCassandraStreamWriter(sstable, header.sections, header.compressionInfo, session); + writer.write(out); } - writer.write(out); } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java index 9d29b2c2e76d..c6dd9a91e6d0 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java @@ -42,7 +42,7 @@ /** * CassandraStreamWriter writes given section of the SSTable to given channel. */ -public class CassandraStreamWriter implements IStreamWriter +public class CassandraStreamWriter { private static final int DEFAULT_CHUNK_SIZE = 64 * 1024; @@ -69,7 +69,6 @@ public CassandraStreamWriter(SSTableReader sstable, Collection Date: Thu, 26 Jul 2018 17:29:05 -0700 Subject: [PATCH 57/64] Assert format --- .../cassandra/db/streaming/CassandraBlockStreamReader.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index ef69d728b22a..b94820d65771 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -32,6 +32,7 @@ import org.apache.cassandra.io.sstable.Component; 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.BigTableBlockWriter; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.schema.TableId; @@ -57,6 +58,9 @@ public class CassandraBlockStreamReader implements IStreamReader public CassandraBlockStreamReader(StreamMessageHeader messageHeader, CassandraStreamHeader streamHeader, StreamSession session) { + if (streamHeader.format != SSTableFormat.Type.BIG) + throw new AssertionError("Unsupported SSTable format " + streamHeader.format); + if (session.getPendingRepair() != null) { // we should only ever be streaming pending repair sstables if the session has a pending repair id From 07a945ac23a5c77cf046cc3b1243042c78653205 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 17:30:01 -0700 Subject: [PATCH 58/64] Stylistic changes --- .../cassandra/db/streaming/CassandraBlockStreamReader.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java index b94820d65771..ee0daad737b1 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java @@ -75,13 +75,13 @@ public CassandraBlockStreamReader(StreamMessageHeader messageHeader, CassandraSt } /** - * @param inputPlus where this reads data from + * @param in where this reads data from * @return SSTable transferred * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails. */ @SuppressWarnings("resource") // input needs to remain open, streams on top of it can't be closed @Override - public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException + public SSTableMultiWriter read(DataInputPlus in) throws IOException { ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId); if (cfs == null) @@ -118,7 +118,7 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException prettyPrintMemory(bytesRead), prettyPrintMemory(totalSize)); - writer.writeComponent(component.type, inputPlus, length); + writer.writeComponent(component.type, in, length); session.progress(writer.descriptor.filenameFor(component), ProgressInfo.Direction.IN, length, length); bytesRead += length; From 285e23b1f813901ac84722f5ad59af2a042ed834 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 26 Jul 2018 17:48:05 -0700 Subject: [PATCH 59/64] Simplify ComponentManifest ser/des --- .../db/streaming/ComponentManifest.java | 6 +- .../cassandra/io/sstable/Component.java | 67 ++++++------------- .../cassandra/io/sstable/ComponentTest.java | 41 ------------ 3 files changed, 22 insertions(+), 92 deletions(-) delete mode 100644 test/unit/org/apache/cassandra/io/sstable/ComponentTest.java diff --git a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java index 2991883457e2..90e3dbd9696a 100644 --- a/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java +++ b/src/java/org/apache/cassandra/db/streaming/ComponentManifest.java @@ -89,7 +89,7 @@ public void serialize(ComponentManifest manifest, DataOutputPlus out, int versio out.writeUnsignedVInt(manifest.components.size()); for (Map.Entry entry : manifest.components.entrySet()) { - out.writeByte(entry.getKey().type.id); + out.writeUTF(entry.getKey().name); out.writeUnsignedVInt(entry.getValue()); } } @@ -102,7 +102,7 @@ public ComponentManifest deserialize(DataInputPlus in, int version) throws IOExc for (int i = 0; i < size; i++) { - Component component = Component.get(Component.Type.fromRepresentation(in.readByte())); + Component component = Component.parse(in.readUTF()); long length = in.readUnsignedVInt(); components.put(component, length); } @@ -115,7 +115,7 @@ public long serializedSize(ComponentManifest manifest, int version) long size = TypeSizes.sizeofUnsignedVInt(manifest.components.size()); for (Map.Entry entry : manifest.components.entrySet()) { - size += TypeSizes.sizeof(entry.getKey().type.id); + size += TypeSizes.sizeof(entry.getKey().name); size += TypeSizes.sizeofUnsignedVInt(entry.getValue()); } return size; diff --git a/src/java/org/apache/cassandra/io/sstable/Component.java b/src/java/org/apache/cassandra/io/sstable/Component.java index 229a2bb3e9e6..435dd0666183 100644 --- a/src/java/org/apache/cassandra/io/sstable/Component.java +++ b/src/java/org/apache/cassandra/io/sstable/Component.java @@ -18,12 +18,9 @@ package org.apache.cassandra.io.sstable; import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; import java.util.regex.Pattern; import com.google.common.base.Objects; -import com.google.common.collect.ImmutableMap; /** * SSTables are made up of multiple components in separate files. Components are @@ -36,40 +33,41 @@ public class Component final static EnumSet TYPES = EnumSet.allOf(Type.class); + /** + * WARNING: Be careful while changing the names or string representation of the enum + * members. Streaming code depends on the names during streaming (Ref: CASSANDRA-14556). + */ public enum Type { // the base data for an sstable: the remaining components can be regenerated // based on the data component - DATA("Data.db", 1), + DATA("Data.db"), // index of the row keys with pointers to their positions in the data file - PRIMARY_INDEX("Index.db", 2), + PRIMARY_INDEX("Index.db"), // serialized bloom filter for the row keys in the sstable - FILTER("Filter.db", 3), + FILTER("Filter.db"), // file to hold information about uncompressed data length, chunk offsets etc. - COMPRESSION_INFO("CompressionInfo.db", 4), + COMPRESSION_INFO("CompressionInfo.db"), // statistical metadata about the content of the sstable - STATS("Statistics.db", 5), + STATS("Statistics.db"), // holds CRC32 checksum of the data file - DIGEST("Digest.crc32", 6), + DIGEST("Digest.crc32"), // holds the CRC32 for chunks in an a uncompressed file. - CRC("CRC.db", 7), + CRC("CRC.db"), // holds SSTable Index Summary (sampling of Index component) - SUMMARY("Summary.db", 8), + SUMMARY("Summary.db"), // table of contents, stores the list of all components for the sstable - TOC("TOC.txt", 9), + TOC("TOC.txt"), // built-in secondary index (may be multiple per sstable) - SECONDARY_INDEX("SI_.*.db", 10), + SECONDARY_INDEX("SI_.*.db"), // custom component, used by e.g. custom compaction strategy - CUSTOM(null, 11); + CUSTOM(null); public final String repr; - public final byte id; - public static final Map idToType; - Type(String repr, int id) + Type(String repr) { this.repr = repr; - this.id = (byte) id; } public static Type fromRepresentation(String repr) @@ -81,21 +79,6 @@ public static Type fromRepresentation(String repr) } return CUSTOM; } - - public static Type fromRepresentation(byte id) - { - return idToType.getOrDefault(id, CUSTOM); - } - - static - { - Type[] values = Type.values(); - Map result = new HashMap<>(values.length); - for (Type t : values) - if (!t.equals(CUSTOM)) result.put(t.id, t); - - idToType = ImmutableMap.copyOf(result); - } } // singleton components for types that don't need ids @@ -145,21 +128,7 @@ public String name() public static Component parse(String name) { Type type = Type.fromRepresentation(name); - Component c = get(type); - - if (c != null) - return c; - - switch (type) - { - case SECONDARY_INDEX: return new Component(Type.SECONDARY_INDEX, name); - case CUSTOM: return new Component(Type.CUSTOM, name); - default: throw new AssertionError(); - } - } - public static Component get(Type type) - { // Build (or retrieve singleton for) the component object switch (type) { @@ -172,7 +141,9 @@ public static Component get(Type type) case CRC: return Component.CRC; case SUMMARY: return Component.SUMMARY; case TOC: return Component.TOC; - default: return null; + case SECONDARY_INDEX: return new Component(Type.SECONDARY_INDEX, name); + case CUSTOM: return new Component(Type.CUSTOM, name); + default: throw new AssertionError(); } } diff --git a/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java b/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java deleted file mode 100644 index 21c58500bd35..000000000000 --- a/test/unit/org/apache/cassandra/io/sstable/ComponentTest.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.io.sstable; - -import org.junit.Test; - -import static org.junit.Assert.assertTrue; - -public class ComponentTest -{ - @Test - public void testTypeCreationFromRepresentation_ValidId() - { - for (int i = 1; i < 10; i++) - assertTrue(Component.Type.fromRepresentation((byte) i) != Component.Type.CUSTOM); - } - - @Test - public void testTypeCreationFromRepresentation_InvalidIds() - { - assertTrue(Component.Type.fromRepresentation((byte) -1) == Component.Type.CUSTOM); - assertTrue(Component.Type.fromRepresentation((byte) 11) == Component.Type.CUSTOM); - assertTrue(Component.Type.fromRepresentation((byte) 12) == Component.Type.CUSTOM); - } -} From 3f9a5814bb0c6a4b8c88ca28a6cd0b8b18831ce0 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 27 Jul 2018 08:14:33 -0700 Subject: [PATCH 60/64] Remove unused import, STCS check. Call into DatabaseDescriptor directly instead of caching value. --- .../db/streaming/CassandraBlockStreamWriter.java | 1 - .../db/streaming/CassandraOutgoingFile.java | 13 ++----------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java index fdf1a422cea2..f87dd01308a3 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java @@ -27,7 +27,6 @@ import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.io.util.DataOutputStreamPlus; import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; import org.apache.cassandra.streaming.ProgressInfo; import org.apache.cassandra.streaming.StreamManager; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 782e50297237..cba4973a2eab 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -56,7 +56,6 @@ */ public class CassandraOutgoingFile implements OutgoingStream { - private static final boolean isZeroCopySSTableTransfersEnabled = DatabaseDescriptor.isZeroCopySSTableTransfersEnabled(); public 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); @@ -184,7 +183,8 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) public boolean shouldStreamEntireSSTable() { // don't stream if full sstable transfers are disabled or legacy counter shards are present - if (!isZeroCopySSTableTransfersEnabled || ref.get().getSSTableMetadata().hasLegacyCounterShards) + if (!DatabaseDescriptor.isZeroCopySSTableTransfersEnabled() + || ref.get().getSSTableMetadata().hasLegacyCounterShards) return false; ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(getTableId()); @@ -198,15 +198,6 @@ public boolean shouldStreamEntireSSTable() if (compactionStrategy instanceof LeveledCompactionStrategy) return contained(ranges, ref.get()); - if (compactionStrategy instanceof SizeTieredCompactionStrategy) - { - return (ranges != null - && ranges.size() == 1 - && ranges.get(0) - .contains(new Range<>(ref.get().first.getToken(), - ref.get().last.getToken()))); - } - return false; } From ad71c3ebad92e7ebc4254b920b831dcb7d0c1b19 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 27 Jul 2018 08:19:47 -0700 Subject: [PATCH 61/64] Better naming --- ...=> CassandraEntireSSTableStreamReader.java} | 8 ++++---- ...=> CassandraEntireSSTableStreamWriter.java} | 8 ++++---- .../db/streaming/CassandraIncomingFile.java | 2 +- .../db/streaming/CassandraOutgoingFile.java | 3 +-- .../microbench/ZeroCopyStreamingBenchmark.java | 18 +++++++++--------- ...assandraEntireSSTableStreamWriterTest.java} | 10 +++++----- 6 files changed, 24 insertions(+), 25 deletions(-) rename src/java/org/apache/cassandra/db/streaming/{CassandraBlockStreamReader.java => CassandraEntireSSTableStreamReader.java} (95%) rename src/java/org/apache/cassandra/db/streaming/{CassandraBlockStreamWriter.java => CassandraEntireSSTableStreamWriter.java} (93%) rename test/unit/org/apache/cassandra/db/streaming/{CassandraBlockStreamWriterTest.java => CassandraEntireSSTableStreamWriterTest.java} (92%) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java similarity index 95% rename from src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java rename to src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java index ee0daad737b1..280e95371031 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamReader.java @@ -45,18 +45,18 @@ import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; /** - * CassandraBlockStreamReader reads SSTable off the wire and writes it to disk. + * CassandraEntireSSTableStreamReader reads SSTable off the wire and writes it to disk. */ -public class CassandraBlockStreamReader implements IStreamReader +public class CassandraEntireSSTableStreamReader implements IStreamReader { - private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamReader.class); + private static final Logger logger = LoggerFactory.getLogger(CassandraEntireSSTableStreamReader.class); private final TableId tableId; private final StreamSession session; private final CassandraStreamHeader header; private final int fileSequenceNumber; - public CassandraBlockStreamReader(StreamMessageHeader messageHeader, CassandraStreamHeader streamHeader, StreamSession session) + public CassandraEntireSSTableStreamReader(StreamMessageHeader messageHeader, CassandraStreamHeader streamHeader, StreamSession session) { if (streamHeader.format != SSTableFormat.Type.BIG) throw new AssertionError("Unsupported SSTable format " + streamHeader.format); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java similarity index 93% rename from src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java rename to src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java index f87dd01308a3..8c36f04f6eaa 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java @@ -36,18 +36,18 @@ import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; /** - * CassandraBlockStreamWriter streams the entire SSTable to given channel. + * CassandraEntireSSTableStreamWriter streams the entire SSTable to given channel. */ -public class CassandraBlockStreamWriter +public class CassandraEntireSSTableStreamWriter { - private static final Logger logger = LoggerFactory.getLogger(CassandraBlockStreamWriter.class); + private static final Logger logger = LoggerFactory.getLogger(CassandraEntireSSTableStreamWriter.class); private final SSTableReader sstable; private final ComponentManifest manifest; private final StreamSession session; private final StreamRateLimiter limiter; - public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession session, ComponentManifest manifest) + public CassandraEntireSSTableStreamWriter(SSTableReader sstable, StreamSession session, ComponentManifest manifest) { this.session = session; this.sstable = sstable; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java index 3c5299415688..4887ea72df22 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java @@ -65,7 +65,7 @@ public synchronized void read(DataInputPlus in, int version) throws IOException IStreamReader reader; if (streamHeader.isEntireSSTable) { - reader = new CassandraBlockStreamReader(header, streamHeader, session); + reader = new CassandraEntireSSTableStreamReader(header, streamHeader, session); } else { diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index cba4973a2eab..70eb9f961c43 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -35,7 +35,6 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.compaction.AbstractCompactionStrategy; import org.apache.cassandra.db.compaction.LeveledCompactionStrategy; -import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.Component; @@ -166,7 +165,7 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) if (shouldStreamEntireSSTable() && out instanceof ByteBufDataOutputStreamPlus) { - CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, manifest); + CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest); writer.write((ByteBufDataOutputStreamPlus) out); } else diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index b61bd1ef356a..3192bccddefd 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -40,8 +40,8 @@ import org.apache.cassandra.db.RowUpdateBuilder; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.compaction.CompactionManager; -import org.apache.cassandra.db.streaming.CassandraBlockStreamReader; -import org.apache.cassandra.db.streaming.CassandraBlockStreamWriter; +import org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader; +import org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamWriter; import org.apache.cassandra.db.streaming.CassandraOutgoingFile; import org.apache.cassandra.db.streaming.CassandraStreamHeader; import org.apache.cassandra.db.streaming.CassandraStreamReader; @@ -102,10 +102,10 @@ public static class BenchmarkState private static SSTableReader sstable; private static ColumnFamilyStore store; private StreamSession session; - private CassandraBlockStreamWriter blockStreamWriter; + private CassandraEntireSSTableStreamWriter blockStreamWriter; private ByteBuf serializedBlockStream; private InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); - private CassandraBlockStreamReader blockStreamReader; + private CassandraEntireSSTableStreamReader blockStreamReader; private CassandraStreamWriter partialStreamWriter; private CassandraStreamReader partialStreamReader; private ByteBuf serializedPartialStream; @@ -119,7 +119,7 @@ public void setupBenchmark() throws IOException sstable = store.getLiveSSTables().iterator().next(); session = setupStreamingSessionForTest(); - blockStreamWriter = new CassandraBlockStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); + blockStreamWriter = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); CapturingNettyChannel blockStreamCaptureChannel = new CapturingNettyChannel(STREAM_SIZE); ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, blockStreamCaptureChannel, 1024 * 1024); @@ -143,10 +143,10 @@ public void setupBenchmark() throws IOException .withTableId(sstable.metadata().id) .build(); - blockStreamReader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, - peer, session.planId(), - 0, 0, 0, - null), entireSSTableStreamHeader, session); + blockStreamReader = new CassandraEntireSSTableStreamReader(new StreamMessageHeader(sstable.metadata().id, + peer, session.planId(), + 0, 0, 0, + null), entireSSTableStreamHeader, session); List> requestedRanges = Arrays.asList(new Range<>(sstable.first.minValue().getToken(), sstable.last.getToken())); partialStreamWriter = new CassandraStreamWriter(sstable, sstable.getPositionsForRanges(requestedRanges), session); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java similarity index 92% rename from test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java rename to test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java index e139dd344844..947f9687f4a0 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraBlockStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java @@ -64,9 +64,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class CassandraBlockStreamWriterTest +public class CassandraEntireSSTableStreamWriterTest { - public static final String KEYSPACE = "CassandraBlockStreamWriterTest"; + public static final String KEYSPACE = "CassandraEntireSSTableStreamWriterTest"; public static final String CF_STANDARD = "Standard1"; public static final String CF_INDEXED = "Indexed1"; public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval"; @@ -111,7 +111,7 @@ public void testBlockWriterOverWire() throws IOException { StreamSession session = setupStreamingSessionForTest(); - CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); + CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); EmbeddedChannel channel = new EmbeddedChannel(); ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); @@ -128,7 +128,7 @@ public void testBlockReadingAndWritingOverWire() throws Exception StreamSession session = setupStreamingSessionForTest(); InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); - CassandraBlockStreamWriter writer = new CassandraBlockStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); + CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed ByteBuf serializedFile = Unpooled.buffer(8192); @@ -153,7 +153,7 @@ public void testBlockReadingAndWritingOverWire() throws Exception .withTableId(sstable.metadata().id) .build(); - CassandraBlockStreamReader reader = new CassandraBlockStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); + CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); SSTableMultiWriter sstableWriter = reader.read(new ByteBufDataInputPlus(serializedFile)); Collection newSstables = sstableWriter.finished(); From c24bb27dd07c785dd4b2ed962fe180172cddbd3d Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 27 Jul 2018 08:26:06 -0700 Subject: [PATCH 62/64] Slight renaming for consistency --- ...reamReader.java => CassandraCompressedStreamReader.java} | 6 +++--- ...reamWriter.java => CassandraCompressedStreamWriter.java} | 6 +++--- .../cassandra/db/streaming/CassandraIncomingFile.java | 2 +- .../cassandra/db/streaming/CassandraOutgoingFile.java | 2 +- .../cassandra/db/streaming/CompressedInputStream.java | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) rename src/java/org/apache/cassandra/db/streaming/{CompressedCassandraStreamReader.java => CassandraCompressedStreamReader.java} (97%) rename src/java/org/apache/cassandra/db/streaming/{CompressedCassandraStreamWriter.java => CassandraCompressedStreamWriter.java} (97%) diff --git a/src/java/org/apache/cassandra/db/streaming/CompressedCassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java similarity index 97% rename from src/java/org/apache/cassandra/db/streaming/CompressedCassandraStreamReader.java rename to src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java index c71edfbe66ed..eb993ff0f8d1 100644 --- a/src/java/org/apache/cassandra/db/streaming/CompressedCassandraStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java @@ -40,13 +40,13 @@ /** * CassandraStreamReader that reads from streamed compressed SSTable */ -public class CompressedCassandraStreamReader extends CassandraStreamReader +public class CassandraCompressedStreamReader extends CassandraStreamReader { - private static final Logger logger = LoggerFactory.getLogger(CompressedCassandraStreamReader.class); + private static final Logger logger = LoggerFactory.getLogger(CassandraCompressedStreamReader.class); protected final CompressionInfo compressionInfo; - public CompressedCassandraStreamReader(StreamMessageHeader header, CassandraStreamHeader streamHeader, StreamSession session) + public CassandraCompressedStreamReader(StreamMessageHeader header, CassandraStreamHeader streamHeader, StreamSession session) { super(header, streamHeader, session); this.compressionInfo = streamHeader.compressionInfo; diff --git a/src/java/org/apache/cassandra/db/streaming/CompressedCassandraStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java similarity index 97% rename from src/java/org/apache/cassandra/db/streaming/CompressedCassandraStreamWriter.java rename to src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java index c5b0c53381b8..3b971f885942 100644 --- a/src/java/org/apache/cassandra/db/streaming/CompressedCassandraStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java @@ -40,15 +40,15 @@ /** * CassandraStreamWriter for compressed SSTable. */ -public class CompressedCassandraStreamWriter extends CassandraStreamWriter +public class CassandraCompressedStreamWriter extends CassandraStreamWriter { private static final int CHUNK_SIZE = 1 << 16; - private static final Logger logger = LoggerFactory.getLogger(CompressedCassandraStreamWriter.class); + private static final Logger logger = LoggerFactory.getLogger(CassandraCompressedStreamWriter.class); private final CompressionInfo compressionInfo; - public CompressedCassandraStreamWriter(SSTableReader sstable, Collection sections, CompressionInfo compressionInfo, StreamSession session) + public CassandraCompressedStreamWriter(SSTableReader sstable, Collection sections, CompressionInfo compressionInfo, StreamSession session) { super(sstable, sections, session); this.compressionInfo = compressionInfo; diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java index 4887ea72df22..5043a98d87dd 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraIncomingFile.java @@ -71,7 +71,7 @@ public synchronized void read(DataInputPlus in, int version) throws IOException { reader = !streamHeader.isCompressed() ? new CassandraStreamReader(header, streamHeader, session) - : new CompressedCassandraStreamReader(header, streamHeader, session); + : new CassandraCompressedStreamReader(header, streamHeader, session); } size = streamHeader.size(); sstable = reader.read(in); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index 70eb9f961c43..9993159df938 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -172,7 +172,7 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) { CassandraStreamWriter writer = (header.compressionInfo == null) ? new CassandraStreamWriter(sstable, header.sections, session) : - new CompressedCassandraStreamWriter(sstable, header.sections, + new CassandraCompressedStreamWriter(sstable, header.sections, header.compressionInfo, session); writer.write(out); } diff --git a/src/java/org/apache/cassandra/db/streaming/CompressedInputStream.java b/src/java/org/apache/cassandra/db/streaming/CompressedInputStream.java index 2f56786868bc..c0278e8e0381 100644 --- a/src/java/org/apache/cassandra/db/streaming/CompressedInputStream.java +++ b/src/java/org/apache/cassandra/db/streaming/CompressedInputStream.java @@ -60,7 +60,7 @@ public class CompressedInputStream extends RebufferingInputStream implements Aut private long bufferOffset = 0; /** - * The current {@link CompressedCassandraStreamReader#sections} offset in the stream. + * The current {@link CassandraCompressedStreamReader#sections} offset in the stream. */ private long current = 0; @@ -98,7 +98,7 @@ public CompressedInputStream(DataInputPlus source, CompressionInfo info, Checksu } /** - * Invoked when crossing into the next stream boundary in {@link CompressedCassandraStreamReader#sections}. + * Invoked when crossing into the next stream boundary in {@link CassandraCompressedStreamReader#sections}. */ public void position(long position) throws IOException { From b6a57ea2b0d77140815ecfc6c8a589964699fcb8 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 27 Jul 2018 08:37:55 -0700 Subject: [PATCH 63/64] Update logging --- .../db/streaming/CassandraEntireSSTableStreamWriter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java index 8c36f04f6eaa..7954d577647f 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java @@ -83,7 +83,7 @@ public void write(ByteBufDataOutputStreamPlus out) throws IOException long length = in.size(); // tracks write progress - logger.debug("[Stream #{}] Block streaming {}.{} gen {} component {} size {}", session.planId(), + logger.debug("[Stream #{}] Streaming {}.{} gen {} component {} size {}", session.planId(), sstable.getKeyspaceName(), sstable.getColumnFamilyName(), sstable.descriptor.generation, @@ -95,7 +95,7 @@ public void write(ByteBufDataOutputStreamPlus out) throws IOException session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesWritten, length); - logger.debug("[Stream #{}] Finished block streaming {}.{} gen {} component {} to {}, xfered = {}, length = {}, totalSize = {}", + logger.debug("[Stream #{}] Finished streaming {}.{} gen {} component {} to {}, xfered = {}, length = {}, totalSize = {}", session.planId(), sstable.getKeyspaceName(), sstable.getColumnFamilyName(), @@ -109,7 +109,7 @@ public void write(ByteBufDataOutputStreamPlus out) throws IOException out.flush(); - logger.debug("[Stream #{}] Finished block streaming sstable {} to {}, xfered = {}, totalSize = {}", + logger.debug("[Stream #{}] Finished streaming sstable {} to {}, xfered = {}, totalSize = {}", session.planId(), sstable.getFilename(), session.peer, From 828a1b89bd4fb7eeeae59fbd8c3d0df7244aff46 Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Fri, 27 Jul 2018 08:54:54 -0700 Subject: [PATCH 64/64] Make naming in cassandra.yaml and database descriptor consistent --- conf/cassandra.yaml | 2 +- src/java/org/apache/cassandra/config/Config.java | 2 +- .../org/apache/cassandra/config/DatabaseDescriptor.java | 6 +++--- test/conf/cassandra.yaml | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 58ab3d1bdb81..76f80315b181 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -793,7 +793,7 @@ sstable_preemptive_open_interval_in_mb: 50 # disabled if internode encryption is enabled. Currently this can be used with # Leveled Compaction. Once CASSANDRA-14586 is fixed other compaction strategies # will benefit as well when used in combination with CASSANDRA-6696. -# streaming_zero_copy_sstables_enabled: false +# stream_entire_sstables: false # Throttles all outbound streaming file transfers on this node to the # given total throughput in Mbps. This is necessary because Cassandra does diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 1831048258f7..da0a79944256 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -380,7 +380,7 @@ public class Config public int block_for_peers_timeout_in_secs = 10; public volatile boolean automatic_sstable_upgrade = false; public volatile int max_concurrent_automatic_sstable_upgrades = 1; - public boolean streaming_zero_copy_sstables_enabled = true; + public boolean stream_entire_sstables = true; public volatile AuditLogOptions audit_logging_options = new AuditLogOptions(); diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 820f1fce19fc..660522adfc6c 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -708,12 +708,12 @@ else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_i "server_encryption_options.internode_encryption = " + conf.server_encryption_options.internode_encryption, false); } - if (conf.streaming_zero_copy_sstables_enabled) + if (conf.stream_entire_sstables) { if (conf.server_encryption_options.enabled || conf.server_encryption_options.optional) { logger.warn("Internode encryption enabled. Disabling zero copy SSTable transfers for streaming."); - conf.streaming_zero_copy_sstables_enabled = false; + conf.stream_entire_sstables = false; } } @@ -2271,7 +2271,7 @@ public static int getStreamingConnectionsPerHost() public static boolean isZeroCopySSTableTransfersEnabled() { - return conf.streaming_zero_copy_sstables_enabled; + return conf.stream_entire_sstables; } public static String getLocalDataCenter() diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml index 71f8c6e7fc8f..3c096372ee0d 100644 --- a/test/conf/cassandra.yaml +++ b/test/conf/cassandra.yaml @@ -46,5 +46,5 @@ enable_user_defined_functions: true enable_scripted_user_defined_functions: true prepared_statements_cache_size_mb: 1 corrupted_tombstone_strategy: exception -streaming_zero_copy_sstables_enabled: true +stream_entire_sstables: true stream_throughput_outbound_megabits_per_sec: 200000000