[ https://issues.apache.org/jira/browse/CASSANDRA-14556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16556361#comment-16556361 ]
ASF GitHub Bot commented on CASSANDRA-14556: -------------------------------------------- Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/239#discussion_r205282784 --- Diff: src/java/org/apache/cassandra/io/sstable/format/big/BigTableBlockWriter.java --- @@ -0,0 +1,221 @@ +/* + * 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.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<Component.Type, SequentialWriter> 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<Component> 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<Component> components) + { + super(descriptor, ImmutableSet.copyOf(components), metadata, + DatabaseDescriptor.getDiskOptimizationStrategy()); + txn.trackNew(this); + this.metadata = metadata; + this.txn = txn; --- End diff -- Removed. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org