[
https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16555914#comment-16555914
]
ASF GitHub Bot commented on CASSANDRA-14556:
--------------------------------------------
Github user iamaleksey commented on a diff in the pull request:
https://github.com/apache/cassandra/pull/239#discussion_r205175204
--- Diff:
src/java/org/apache/cassandra/db/streaming/CassandraBlockStreamWriter.java ---
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.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);
+
+ private final SSTableReader sstable;
+ private final ComponentManifest manifest;
+ private final StreamSession session;
+ private final StreamRateLimiter limiter;
+
+ public CassandraBlockStreamWriter(SSTableReader sstable, StreamSession
session, ComponentManifest manifest)
+ {
+ this.session = session;
+ this.sstable = sstable;
+ this.manifest = manifest;
+ this.limiter = StreamManager.getRateLimiter(session.peer);
+ }
+
+ /**
+ * Stream the entire file to given channel.
+ * <p>
+ *
+ * @param output where this writes data to
+ * @throws IOException on any I/O error
+ */
+ @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 progress = 0L;
+ ByteBufDataOutputStreamPlus byteBufDataOutputStreamPlus =
(ByteBufDataOutputStreamPlus) output;
+
+ 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), "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, component,
length);
+
+ bytesRead += byteBufDataOutputStreamPlus.writeToChannel(in,
limiter);
--- End diff --
So this should, essentially, be same as `length`, correct?
> Optimize streaming path in Cassandra
> ------------------------------------
>
> Key: CASSANDRA-14556
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14556
> Project: Cassandra
> Issue Type: Improvement
> Components: Streaming and Messaging
> Reporter: Dinesh Joshi
> Assignee: Dinesh Joshi
> Priority: Major
> Labels: Performance
> Fix For: 4.x
>
>
> During streaming, Cassandra reifies the sstables into objects. This creates
> unnecessary garbage and slows down the whole streaming process as some
> sstables can be transferred as a whole file rather than individual
> partitions. The objective of the ticket is to detect when a whole sstable can
> be transferred and skip the object reification. We can also use a zero-copy
> path to avoid bringing data into user-space on both sending and receiving
> side.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]