You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/06/20 19:07:13 UTC
[1/5] Streaming 2.0
Updated Branches:
refs/heads/trunk 40b6c5d9c -> 515116972
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/SerializationsTest.java b/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
deleted file mode 100644
index 6db5b15..0000000
--- a/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
+++ /dev/null
@@ -1,220 +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.streaming;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
-
-import org.junit.Test;
-
-import org.apache.cassandra.AbstractSerializationsTester;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.dht.BytesToken;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDGen;
-
-public class SerializationsTest extends AbstractSerializationsTester
-{
- private void testPendingFileWrite() throws IOException
- {
- // make sure to test serializing null and a pf with no sstable.
- PendingFile normal = makePendingFile(true, 100, OperationType.BOOTSTRAP);
- PendingFile noSections = makePendingFile(true, 0, OperationType.AES);
- PendingFile noSST = makePendingFile(false, 100, OperationType.RESTORE_REPLICA_COUNT);
-
- DataOutputStream out = getOutput("streaming.PendingFile.bin");
- PendingFile.serializer.serialize(normal, out, getVersion());
- PendingFile.serializer.serialize(noSections, out, getVersion());
- PendingFile.serializer.serialize(noSST, out, getVersion());
- PendingFile.serializer.serialize(null, out, getVersion());
- out.close();
-
- // test serializedSize
- testSerializedSize(normal, PendingFile.serializer);
- testSerializedSize(noSections, PendingFile.serializer);
- testSerializedSize(noSST, PendingFile.serializer);
- testSerializedSize(null, PendingFile.serializer);
- }
-
- @Test
- public void testPendingFileRead() throws IOException
- {
- if (EXECUTE_WRITES)
- testPendingFileWrite();
-
- DataInputStream in = getInput("streaming.PendingFile.bin");
- assert PendingFile.serializer.deserialize(in, getVersion()) != null;
- assert PendingFile.serializer.deserialize(in, getVersion()) != null;
- assert PendingFile.serializer.deserialize(in, getVersion()) != null;
- assert PendingFile.serializer.deserialize(in, getVersion()) == null;
- in.close();
- }
-
- private void testStreamHeaderWrite() throws IOException
- {
- UUID sessionId = UUIDGen.getTimeUUID();
- StreamHeader sh0 = new StreamHeader("Keyspace1", sessionId, makePendingFile(true, 100, OperationType.BOOTSTRAP));
- StreamHeader sh1 = new StreamHeader("Keyspace1", sessionId, makePendingFile(false, 100, OperationType.BOOTSTRAP));
- Collection<PendingFile> files = new ArrayList<PendingFile>();
- for (int i = 0; i < 50; i++)
- files.add(makePendingFile(i % 2 == 0, 100, OperationType.BOOTSTRAP));
- StreamHeader sh2 = new StreamHeader("Keyspace1", sessionId, makePendingFile(true, 100, OperationType.BOOTSTRAP), files);
- StreamHeader sh3 = new StreamHeader("Keyspace1", sessionId, null, files);
- StreamHeader sh4 = new StreamHeader("Keyspace1", sessionId, makePendingFile(true, 100, OperationType.BOOTSTRAP), new ArrayList<PendingFile>());
-
- DataOutputStream out = getOutput("streaming.StreamHeader.bin");
- StreamHeader.serializer.serialize(sh0, out, getVersion());
- StreamHeader.serializer.serialize(sh1, out, getVersion());
- StreamHeader.serializer.serialize(sh2, out, getVersion());
- StreamHeader.serializer.serialize(sh3, out, getVersion());
- StreamHeader.serializer.serialize(sh4, out, getVersion());
- out.close();
-
- // test serializedSize
- testSerializedSize(sh0, StreamHeader.serializer);
- testSerializedSize(sh1, StreamHeader.serializer);
- testSerializedSize(sh2, StreamHeader.serializer);
- testSerializedSize(sh3, StreamHeader.serializer);
- testSerializedSize(sh4, StreamHeader.serializer);
- }
-
- @Test
- public void testStreamHeaderRead() throws IOException
- {
- if (EXECUTE_WRITES)
- testStreamHeaderWrite();
-
- DataInputStream in = getInput("streaming.StreamHeader.bin");
- assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
- assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
- assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
- assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
- assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
- in.close();
- }
-
- private void testStreamReplyWrite() throws IOException
- {
- UUID sessionId = UUIDGen.getTimeUUID();
- StreamReply rep = new StreamReply("this is a file", sessionId, StreamReply.Status.FILE_FINISHED);
- DataOutputStream out = getOutput("streaming.StreamReply.bin");
- StreamReply.serializer.serialize(rep, out, getVersion());
- rep.createMessage().serialize(out, getVersion());
- out.close();
-
- // test serializedSize
- testSerializedSize(rep, StreamReply.serializer);
- }
-
- @Test
- public void testStreamReplyRead() throws IOException
- {
- if (EXECUTE_WRITES)
- testStreamReplyWrite();
-
- DataInputStream in = getInput("streaming.StreamReply.bin");
- assert StreamReply.serializer.deserialize(in, getVersion()) != null;
- assert MessageIn.read(in, getVersion(), -1) != null;
- in.close();
- }
-
- private static PendingFile makePendingFile(boolean sst, int numSecs, OperationType op)
- {
- Descriptor desc = new Descriptor("z", new File("path/doesn't/matter"), "Keyspace1", "Standard1", 23, false);
- List<Pair<Long, Long>> sections = new ArrayList<Pair<Long, Long>>();
- for (int i = 0; i < numSecs; i++)
- sections.add(Pair.create(new Long(i), new Long(i * i)));
- return new PendingFile(sst ? makeSSTable() : null, desc, SSTable.COMPONENT_DATA, sections, op);
- }
-
- private void testStreamRequestMessageWrite() throws IOException
- {
- UUID sessionId = UUIDGen.getTimeUUID();
- Collection<Range<Token>> ranges = new ArrayList<Range<Token>>();
- for (int i = 0; i < 5; i++)
- ranges.add(new Range<Token>(new BytesToken(ByteBufferUtil.bytes(Integer.toString(10*i))), new BytesToken(ByteBufferUtil.bytes(Integer.toString(10*i+5)))));
- List<ColumnFamilyStore> stores = Collections.singletonList(Table.open("Keyspace1").getColumnFamilyStore("Standard1"));
- StreamRequest msg0 = new StreamRequest(FBUtilities.getBroadcastAddress(), ranges, "Keyspace1", stores, sessionId, OperationType.RESTORE_REPLICA_COUNT);
- StreamRequest msg1 = new StreamRequest(FBUtilities.getBroadcastAddress(), makePendingFile(true, 100, OperationType.BOOTSTRAP), sessionId);
- StreamRequest msg2 = new StreamRequest(FBUtilities.getBroadcastAddress(), makePendingFile(false, 100, OperationType.BOOTSTRAP), sessionId);
-
- DataOutputStream out = getOutput("streaming.StreamRequestMessage.bin");
- StreamRequest.serializer.serialize(msg0, out, getVersion());
- StreamRequest.serializer.serialize(msg1, out, getVersion());
- StreamRequest.serializer.serialize(msg2, out, getVersion());
- msg0.createMessage().serialize(out, getVersion());
- msg1.createMessage().serialize(out, getVersion());
- msg2.createMessage().serialize(out, getVersion());
- out.close();
-
- // test serializedSize
- testSerializedSize(msg0, StreamRequest.serializer);
- testSerializedSize(msg1, StreamRequest.serializer);
- testSerializedSize(msg2, StreamRequest.serializer);
- }
-
- @Test
- public void testStreamRequestMessageRead() throws IOException
- {
- if (EXECUTE_WRITES)
- testStreamRequestMessageWrite();
-
- DataInputStream in = getInput("streaming.StreamRequestMessage.bin");
- assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
- assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
- assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
- assert MessageIn.read(in, getVersion(), -1) != null;
- assert MessageIn.read(in, getVersion(), -1) != null;
- assert MessageIn.read(in, getVersion(), -1) != null;
- in.close();
- }
-
- private static SSTableReader makeSSTable()
- {
- Table t = Table.open("Keyspace1");
- for (int i = 0; i < 100; i++)
- {
- RowMutation rm = new RowMutation(t.getName(), ByteBufferUtil.bytes(Long.toString(System.nanoTime())));
- rm.add("Standard1", ByteBufferUtil.bytes("cola"), ByteBufferUtil.bytes("value"), 0);
- rm.apply();
- }
- try
- {
- t.getColumnFamilyStore("Standard1").forceBlockingFlush();
- return t.getColumnFamilyStore("Standard1").getSSTables().iterator().next();
- }
- catch (Exception any)
- {
- throw new RuntimeException(any);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
new file mode 100644
index 0000000..60fbf40
--- /dev/null
+++ b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.FBUtilities;
+
+public class SessionInfoTest
+{
+ /**
+ * Test if total numbers are collect
+ */
+ @Test
+ public void testTotals()
+ {
+ UUID cfId = UUID.randomUUID();
+ InetAddress local = FBUtilities.getLocalAddress();
+
+ Collection<StreamSummary> summaries = new ArrayList<>();
+ for (int i = 0; i < 10; i++)
+ {
+ StreamSummary summary = new StreamSummary(cfId, i, (i + 1) * 10);
+ summaries.add(summary);
+ }
+
+ StreamSummary sending = new StreamSummary(cfId, 10, 100);
+ SessionInfo info = new SessionInfo(local, summaries, Collections.singleton(sending), StreamSession.State.PREPARING);
+
+ assert info.getTotalFilesToReceive() == 45;
+ assert info.getTotalFilesToSend() == 10;
+ assert info.getTotalSizeToReceive() == 550;
+ assert info.getTotalSizeToSend() == 100;
+ // still, no files received or sent
+ assert info.getTotalFilesReceived() == 0;
+ assert info.getTotalFilesSent() == 0;
+
+ // receive in progress
+ info.updateProgress(new ProgressInfo(local, "test.txt", ProgressInfo.Direction.IN, 50, 100));
+ // still in progress, but not completed yet
+ assert info.getTotalSizeReceived() == 50;
+ assert info.getTotalSizeSent() == 0;
+ assert info.getTotalFilesReceived() == 0;
+ assert info.getTotalFilesSent() == 0;
+ info.updateProgress(new ProgressInfo(local, "test.txt", ProgressInfo.Direction.IN, 100, 100));
+ // 1 file should be completed
+ assert info.getTotalSizeReceived() == 100;
+ assert info.getTotalSizeSent() == 0;
+ assert info.getTotalFilesReceived() == 1;
+ assert info.getTotalFilesSent() == 0;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/StreamUtil.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamUtil.java b/test/unit/org/apache/cassandra/streaming/StreamUtil.java
deleted file mode 100644
index 4987923..0000000
--- a/test/unit/org/apache/cassandra/streaming/StreamUtil.java
+++ /dev/null
@@ -1,46 +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.streaming;
-
-import java.io.IOError;
-import java.io.IOException;
-import java.net.InetAddress;
-
-import org.apache.cassandra.net.MessageIn;
-
-public class StreamUtil
-{
- /**
- * Takes an stream request message and creates an empty status response. Exists here because StreamRequestMessage
- * is package protected.
- */
- static public void finishStreamRequest(MessageIn<StreamRequest> msg, InetAddress to)
- {
- StreamInSession session = StreamInSession.get(to, msg.payload.sessionId);
- try
- {
- session.closeIfFinished();
- }
- catch (IOException e)
- {
- throw new IOError(e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index a86330c..3e9eeb1 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -1,35 +1,40 @@
-package org.apache.cassandra.streaming;
-
/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements. See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership. The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License. You may obtain a copy of the License at
-*
-* http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied. See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-
-import static junit.framework.Assert.assertEquals;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.Util;
-import static org.apache.cassandra.Util.column;
+ * 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.streaming;
import java.net.InetAddress;
-import java.sql.Date;
import java.nio.ByteBuffer;
+import java.sql.Date;
import java.util.*;
+import java.util.concurrent.TimeUnit;
+import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
@@ -39,21 +44,18 @@ import org.apache.cassandra.db.filter.QueryFilter;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableUtils;
import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.SSTableUtils;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.thrift.IndexExpression;
import org.apache.cassandra.thrift.IndexOperator;
+import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.CounterId;
import org.apache.cassandra.utils.FBUtilities;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.fail;
+import static org.apache.cassandra.Util.column;
@RunWith(OrderedJUnit4ClassRunner.class)
public class StreamingTransferTest extends SchemaLoader
@@ -69,10 +71,64 @@ public class StreamingTransferTest extends SchemaLoader
}
/**
+ * Test if empty {@link StreamPlan} returns success with empty result.
+ */
+ @Test
+ public void testEmptyStreamPlan() throws Exception
+ {
+ StreamResultFuture futureResult = new StreamPlan("StreamingTransferTest").execute();
+ final UUID planId = futureResult.planId;
+ Futures.addCallback(futureResult, new FutureCallback<StreamState>()
+ {
+ public void onSuccess(StreamState result)
+ {
+ assert planId.equals(result.planId);
+ assert result.description.equals("StreamingTransferTest");
+ assert result.sessions.isEmpty();
+ }
+
+ public void onFailure(Throwable t)
+ {
+ fail();
+ }
+ });
+ // should be complete immediately
+ futureResult.get(100, TimeUnit.MILLISECONDS);
+ }
+
+ @Test
+ public void testRequestEmpty() throws Exception
+ {
+ // requesting empty data should succeed
+ IPartitioner p = StorageService.getPartitioner();
+ List<Range<Token>> ranges = new ArrayList<>();
+ 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("StreamingTransferTest")
+ .requestRanges(LOCAL, "Keyspace2", ranges)
+ .execute();
+
+ UUID planId = futureResult.planId;
+ StreamState result = futureResult.get();
+ assert planId.equals(result.planId);
+ assert result.description.equals("StreamingTransferTest");
+
+ // we should have completed session with empty transfer
+ assert result.sessions.size() == 1;
+ SessionInfo session = Iterables.get(result.sessions, 0);
+ assert session.peer.equals(LOCAL);
+ assert session.getTotalFilesReceived() == 0;
+ assert session.getTotalFilesSent() == 0;
+ assert session.getTotalSizeReceived() == 0;
+ assert session.getTotalSizeSent() == 0;
+ }
+
+ /**
* Create and transfer a single sstable, and return the keys that should have been transferred.
* The Mutator must create the given column, but it may also create any other columns it pleases.
*/
- private List<String> createAndTransfer(Table table, ColumnFamilyStore cfs, Mutator mutator) throws Exception
+ private List<String> createAndTransfer(ColumnFamilyStore cfs, Mutator mutator) throws Exception
{
// write a temporary SSTable, and unregister it
logger.debug("Mutating " + cfs.name);
@@ -87,7 +143,7 @@ public class StreamingTransferTest extends SchemaLoader
// transfer the first and last key
logger.debug("Transferring " + cfs.name);
- transfer(table, sstable);
+ transfer(sstable);
// confirm that a single SSTable was transferred and registered
assertEquals(1, cfs.getSSTables().size());
@@ -108,7 +164,7 @@ public class StreamingTransferTest extends SchemaLoader
// and that the max timestamp for the file was rediscovered
assertEquals(timestamp, cfs.getSSTables().iterator().next().getMaxTimestamp());
- List<String> keys = new ArrayList<String>();
+ List<String> keys = new ArrayList<>();
for (int off : offs)
keys.add("key" + off);
@@ -116,20 +172,18 @@ public class StreamingTransferTest extends SchemaLoader
return keys;
}
- private void transfer(Table table, SSTableReader sstable) throws Exception
+ private void transfer(SSTableReader sstable) throws Exception
{
IPartitioner p = StorageService.getPartitioner();
- List<Range<Token>> ranges = new ArrayList<Range<Token>>();
- ranges.add(new Range<Token>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("key1"))));
- ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
- transfer(table, sstable, ranges);
+ List<Range<Token>> ranges = new ArrayList<>();
+ ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("key1"))));
+ ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
+ transfer(sstable, ranges);
}
- private void transfer(Table table, SSTableReader sstable, List<Range<Token>> ranges) throws Exception
+ private void transfer(SSTableReader sstable, List<Range<Token>> ranges) throws Exception
{
- StreamOutSession session = StreamOutSession.create(table.getName(), LOCAL, (IStreamCallback)null);
- StreamOut.transferSSTables(session, Arrays.asList(sstable), ranges, OperationType.BOOTSTRAP);
- session.await();
+ new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, ranges, Collections.singleton(sstable)).execute().get();
}
/**
@@ -157,7 +211,7 @@ public class StreamingTransferTest extends SchemaLoader
SSTableReader sstable = cfs.getSSTables().iterator().next();
cfs.clearUnsafe();
- transfer(table, sstable);
+ transfer(sstable);
// confirm that a single SSTable was transferred and registered
assertEquals(1, cfs.getSSTables().size());
@@ -172,7 +226,7 @@ public class StreamingTransferTest extends SchemaLoader
final Table table = Table.open("Keyspace1");
final ColumnFamilyStore cfs = table.getColumnFamilyStore("Indexed1");
- List<String> keys = createAndTransfer(table, cfs, new Mutator()
+ List<String> keys = createAndTransfer(cfs, new Mutator()
{
public void mutate(String key, String col, long timestamp) throws Exception
{
@@ -190,7 +244,6 @@ public class StreamingTransferTest extends SchemaLoader
for (String key : keys)
{
long val = key.hashCode();
- IPartitioner p = StorageService.getPartitioner();
IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"),
IndexOperator.EQ,
ByteBufferUtil.bytes(val));
@@ -210,14 +263,14 @@ public class StreamingTransferTest extends SchemaLoader
final ColumnFamilyStore cfs = table.getColumnFamilyStore("Counter1");
final CounterContext cc = new CounterContext();
- final Map<String, ColumnFamily> cleanedEntries = new HashMap<String, ColumnFamily>();
+ final Map<String, ColumnFamily> cleanedEntries = new HashMap<>();
- List<String> keys = createAndTransfer(table, cfs, new Mutator()
+ List<String> keys = createAndTransfer(cfs, new Mutator()
{
/** Creates a new SSTable per key: all will be merged before streaming. */
public void mutate(String key, String col, long timestamp) throws Exception
{
- Map<String, ColumnFamily> entries = new HashMap<String, ColumnFamily>();
+ Map<String, ColumnFamily> entries = new HashMap<>();
ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.metadata);
ColumnFamily cfCleaned = TreeMapBackedSortedColumns.factory.create(cfs.metadata);
CounterContext.ContextState state = CounterContext.ContextState.allocate(4, 1);
@@ -254,7 +307,7 @@ public class StreamingTransferTest extends SchemaLoader
// Retransfer the file, making sure it is now idempotent (see CASSANDRA-3481)
cfs.clearUnsafe();
- transfer(table, streamed);
+ transfer(streamed);
SSTableReader restreamed = cfs.getSSTables().iterator().next();
SSTableUtils.assertContentEquals(streamed, restreamed);
}
@@ -263,7 +316,7 @@ public class StreamingTransferTest extends SchemaLoader
public void testTransferTableMultiple() throws Exception
{
// write temporary SSTables, but don't register them
- Set<String> content = new HashSet<String>();
+ Set<String> content = new HashSet<>();
content.add("test");
content.add("test2");
content.add("test3");
@@ -271,7 +324,7 @@ public class StreamingTransferTest extends SchemaLoader
String tablename = sstable.getTableName();
String cfname = sstable.getColumnFamilyName();
- content = new HashSet<String>();
+ content = new HashSet<>();
content.add("transfer1");
content.add("transfer2");
content.add("transfer3");
@@ -279,15 +332,13 @@ public class StreamingTransferTest extends SchemaLoader
// transfer the first and last key
IPartitioner p = StorageService.getPartitioner();
- List<Range<Token>> ranges = new ArrayList<Range<Token>>();
- ranges.add(new Range<Token>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("test"))));
- ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("transfer2")), p.getMinimumToken()));
+ List<Range<Token>> ranges = new ArrayList<>();
+ ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("test"))));
+ ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("transfer2")), p.getMinimumToken()));
// Acquiring references, transferSSTables needs it
sstable.acquireReference();
sstable2.acquireReference();
- StreamOutSession session = StreamOutSession.create(tablename, LOCAL, (IStreamCallback) null);
- StreamOut.transferSSTables(session, Arrays.asList(sstable, sstable2), ranges, OperationType.BOOTSTRAP);
- session.await();
+ new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, ranges, Arrays.asList(sstable, sstable2)).execute().get();
// confirm that the sstables were transferred and registered and that 2 keys arrived
ColumnFamilyStore cfstore = Table.open(tablename).getColumnFamilyStore(cfname);
@@ -311,12 +362,12 @@ public class StreamingTransferTest extends SchemaLoader
String keyspace = "KeyCacheSpace";
IPartitioner p = StorageService.getPartitioner();
String[] columnFamilies = new String[] { "Standard1", "Standard2", "Standard3" };
- List<SSTableReader> ssTableReaders = new ArrayList<SSTableReader>();
+ List<SSTableReader> ssTableReaders = new ArrayList<>();
- NavigableMap<DecoratedKey,String> keys = new TreeMap<DecoratedKey,String>();
+ NavigableMap<DecoratedKey,String> keys = new TreeMap<>();
for (String cf : columnFamilies)
{
- Set<String> content = new HashSet<String>();
+ Set<String> content = new HashSet<>();
content.add("data-" + cf + "-1");
content.add("data-" + cf + "-2");
content.add("data-" + cf + "-3");
@@ -332,19 +383,16 @@ public class StreamingTransferTest extends SchemaLoader
Map.Entry<DecoratedKey,String> first = keys.firstEntry();
Map.Entry<DecoratedKey,String> last = keys.lastEntry();
Map.Entry<DecoratedKey,String> secondtolast = keys.lowerEntry(last.getKey());
- List<Range<Token>> ranges = new ArrayList<Range<Token>>();
- ranges.add(new Range<Token>(p.getMinimumToken(), first.getKey().token));
+ List<Range<Token>> ranges = new ArrayList<>();
+ ranges.add(new Range<>(p.getMinimumToken(), first.getKey().token));
// the left hand side of the range is exclusive, so we transfer from the second-to-last token
- ranges.add(new Range<Token>(secondtolast.getKey().token, p.getMinimumToken()));
+ ranges.add(new Range<>(secondtolast.getKey().token, p.getMinimumToken()));
// Acquiring references, transferSSTables needs it
if (!SSTableReader.acquireReferences(ssTableReaders))
throw new AssertionError();
- StreamOutSession session = StreamOutSession.create(keyspace, LOCAL, (IStreamCallback)null);
- StreamOut.transferSSTables(session, ssTableReaders, ranges, OperationType.BOOTSTRAP);
-
- session.await();
+ new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, ranges, ssTableReaders).execute().get();
// check that only two keys were transferred
for (Map.Entry<DecoratedKey,String> entry : Arrays.asList(first, last))
@@ -382,11 +430,11 @@ public class StreamingTransferTest extends SchemaLoader
cfs.clearUnsafe();
IPartitioner p = StorageService.getPartitioner();
- List<Range<Token>> ranges = new ArrayList<Range<Token>>();
- ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key1000"))));
- ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key5")), p.getToken(ByteBufferUtil.bytes("key500"))));
- ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key9")), p.getToken(ByteBufferUtil.bytes("key900"))));
- transfer(table, sstable, ranges);
+ List<Range<Token>> ranges = new ArrayList<>();
+ ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key1000"))));
+ ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key5")), p.getToken(ByteBufferUtil.bytes("key500"))));
+ ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key9")), p.getToken(ByteBufferUtil.bytes("key900"))));
+ transfer(sstable, ranges);
assertEquals(1, cfs.getSSTables().size());
assertEquals(7, Util.getRangeSlice(cfs).size());
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
index 95297b1..ab311e6 100644
--- a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
@@ -26,7 +26,10 @@ import java.util.*;
import org.junit.Test;
import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.io.compress.*;
+import org.apache.cassandra.io.compress.CompressedSequentialWriter;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.io.compress.SnappyCompressor;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.SSTableMetadata;
[2/5] Streaming 2.0
Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingService.java b/src/java/org/apache/cassandra/streaming/StreamingService.java
deleted file mode 100644
index ea22f82..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamingService.java
+++ /dev/null
@@ -1,110 +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.streaming;
-
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-
-public class StreamingService implements StreamingServiceMBean
-{
- public static final String MBEAN_OBJECT_NAME = "org.apache.cassandra.net:type=StreamingService";
- public static final StreamingService instance = new StreamingService();
-
- private StreamingService()
- {
- MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
- try
- {
- mbs.registerMBean(this, new ObjectName(MBEAN_OBJECT_NAME));
- }
- catch (Exception e)
- {
- throw new RuntimeException(e);
- }
- }
-
- public String getStatus()
- {
- StringBuilder sb = new StringBuilder();
- sb.append("Receiving from:\n");
- for (InetAddress source : StreamInSession.getSources())
- {
- sb.append(String.format(" %s:%n", source.getHostAddress()));
- for (PendingFile pf : StreamInSession.getIncomingFiles(source))
- {
- sb.append(String.format(" %s%n", pf.toString()));
- }
- }
- sb.append("Sending to:%n");
- for (InetAddress dest : StreamOutSession.getDestinations())
- {
- sb.append(String.format(" %s:%n", dest.getHostAddress()));
- for (PendingFile pf : StreamOutSession.getOutgoingFiles(dest))
- {
- sb.append(String.format(" %s%n", pf.toString()));
- }
- }
- return sb.toString();
- }
-
- /** hosts receiving outgoing streams. */
- public Set<InetAddress> getStreamDestinations()
- {
- return StreamOutSession.getDestinations();
- }
-
- /** outgoing streams */
- public List<String> getOutgoingFiles(String host) throws IOException
- {
- List<String> files = new ArrayList<String>();
- // first, verify that host is a destination. calling StreamOutManager.get will put it in the collection
- // leading to false positives in the future.
- Set<InetAddress> existingDestinations = getStreamDestinations();
- InetAddress dest = InetAddress.getByName(host);
- if (!existingDestinations.contains(dest))
- return files;
-
- for (PendingFile f : StreamOutSession.getOutgoingFiles(dest))
- files.add(String.format("%s", f.toString()));
- return files;
- }
-
- /** hosts sending incoming streams */
- public Set<InetAddress> getStreamSources()
- {
- return StreamInSession.getSources();
- }
-
- /** details about incoming streams. */
- public List<String> getIncomingFiles(String host) throws IOException
- {
- List<String> files = new ArrayList<String>();
- for (PendingFile pf : StreamInSession.getIncomingFiles(InetAddress.getByName(host)))
- {
- files.add(String.format("%s: %s", pf.desc.ksname, pf.toString()));
- }
- return files;
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java b/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
deleted file mode 100644
index 8d748f3..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.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.streaming;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.List;
-import java.util.Set;
-
-public interface StreamingServiceMBean
-{
- /** hosts recieving outgoing streams */
- public Set<InetAddress> getStreamDestinations();
-
- /** outgoing streams */
- public List<String> getOutgoingFiles(String host) throws IOException;
-
- /** hosts sending incoming streams. */
- public Set<InetAddress> getStreamSources();
-
- /** details about incoming streams */
- public List<String> getIncomingFiles(String host) throws IOException;
-
- /** What's currently happening wrt streaming. */
- public String getStatus();
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java b/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java
deleted file mode 100644
index 07fb765..0000000
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java
+++ /dev/null
@@ -1,160 +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.streaming.compress;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.metrics.StreamingMetrics;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.streaming.FileStreamTask;
-import org.apache.cassandra.streaming.StreamHeader;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
-
-/**
- * FileStreamTask for compressed SSTable.
- *
- * This class sends relevant part of compressed file directly using nio if available.
- */
-public class CompressedFileStreamTask extends FileStreamTask
-{
- private static final Logger logger = LoggerFactory.getLogger(CompressedFileStreamTask.class);
- // 10MB chunks
- public static final int CHUNK_SIZE = 10*1024*1024;
-
- public CompressedFileStreamTask(StreamHeader header, InetAddress to)
- {
- super(header, to);
- }
-
- protected void stream() throws IOException
- {
- assert header.file.compressionInfo != null;
-
- SocketChannel sc = socket.getChannel();
- byte[] transferBuffer = null;
-
- // write header
- ByteBuffer headerBuffer = MessagingService.instance().constructStreamHeader(header, false, MessagingService.instance().getVersion(to));
- socket.getOutputStream().write(ByteBufferUtil.getArray(headerBuffer));
-
- RandomAccessReader file = RandomAccessReader.open(new File(header.file.getFilename()));
- FileChannel fc = file.getChannel();
-
- StreamingMetrics.activeStreamsOutbound.inc();
- // calculate chunks to transfer. we want to send continuous chunks altogether.
- List<Pair<Long, Long>> sections = getTransferSections(header.file.compressionInfo.chunks);
- try
- {
- long totalBytesTransferred = 0;
- // stream each of the required sections of the file
- for (Pair<Long, Long> section : sections)
- {
- // seek to the beginning of the section when socket channel is not available
- if (sc == null)
- file.seek(section.left);
- // length of the section to stream
- long length = section.right - section.left;
- // tracks write progress
- long bytesTransferred = 0;
- while (bytesTransferred < length)
- {
- int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
- long lastWrite;
- if (sc != null)
- {
- lastWrite = fc.transferTo(section.left + bytesTransferred, toTransfer, sc);
- throttle.throttleDelta(lastWrite);
- }
- else
- {
- // NIO is not available. Fall back to normal streaming.
- // This happens when inter-node encryption is turned on.
- if (transferBuffer == null)
- transferBuffer = new byte[CHUNK_SIZE];
- file.readFully(transferBuffer, 0, toTransfer);
- socket.getOutputStream().write(transferBuffer, 0, toTransfer);
- throttle.throttleDelta(toTransfer);
- lastWrite = toTransfer;
- }
- totalBytesTransferred += lastWrite;
- bytesTransferred += lastWrite;
- header.file.progress += lastWrite;
- }
-
- if (sc == null)
- socket.getOutputStream().flush();
-
- logger.debug("Bytes transferred " + bytesTransferred + "/" + header.file.size);
- }
- StreamingMetrics.totalOutgoingBytes.inc(totalBytesTransferred);
- metrics.outgoingBytes.inc(totalBytesTransferred);
- // receive reply confirmation
- receiveReply();
- }
- finally
- {
- StreamingMetrics.activeStreamsOutbound.dec();
-
- // no matter what happens close file
- FileUtils.closeQuietly(file);
- }
- }
-
- // chunks are assumed to be sorted by offset
- private List<Pair<Long, Long>> getTransferSections(CompressionMetadata.Chunk[] chunks)
- {
- List<Pair<Long, Long>> transferSections = new ArrayList<Pair<Long, Long>>();
- Pair<Long, Long> lastSection = null;
- for (CompressionMetadata.Chunk chunk : chunks)
- {
- if (lastSection != null)
- {
- if (chunk.offset == lastSection.right)
- {
- // extend previous section to end of this chunk
- lastSection = Pair.create(lastSection.left, chunk.offset + chunk.length + 4); // 4 bytes for CRC
- }
- else
- {
- transferSections.add(lastSection);
- lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
- }
- }
- else
- {
- lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
- }
- }
- if (lastSection != null)
- transferSections.add(lastSection);
- return transferSections;
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java b/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
index 6fe1793..3305f50 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
@@ -17,9 +17,12 @@
*/
package org.apache.cassandra.streaming.compress;
-import java.io.*;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
import java.util.Iterator;
-import java.util.concurrent.*;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
import java.util.zip.CRC32;
import java.util.zip.Checksum;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
new file mode 100644
index 0000000..f3e511b
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.compress;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamReader;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.messages.FileMessageHeader;
+import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamReader that reads from streamed compressed SSTable
+ */
+public class CompressedStreamReader extends StreamReader
+{
+ protected final CompressionInfo compressionInfo;
+
+ public CompressedStreamReader(FileMessageHeader header, StreamSession session)
+ {
+ super(header, session);
+ this.compressionInfo = header.compressionInfo;
+ }
+
+ /**
+ * @return SSTable transferred
+ * @throws java.io.IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
+ */
+ @Override
+ public SSTableReader read(ReadableByteChannel channel) throws IOException
+ {
+ long totalSize = totalSize();
+ CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo);
+
+ Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+ Directories.DataDirectory localDir = cfs.directories.getLocationCapableOfSize(totalSize);
+ if (localDir == null)
+ throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
+ desc = Descriptor.fromFilename(cfs.getTempSSTablePath(cfs.directories.getLocationForDisk(localDir)));
+
+ SSTableWriter writer = new SSTableWriter(desc.filenameFor(Component.DATA), estimatedKeys);
+ try
+ {
+ BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));
+
+ for (Pair<Long, Long> section : sections)
+ {
+ long length = section.right - section.left;
+ // skip to beginning of section inside chunk
+ cis.position(section.left);
+ in.reset(0);
+ while (in.getBytesRead() < length)
+ {
+ writeRow(writer, in, cfs);
+ // when compressed, report total bytes of compressed chunks read since remoteFile.size is the sum of chunks transferred
+ session.progress(desc, ProgressInfo.Direction.IN, cis.getTotalCompressedBytesRead(), totalSize);
+ }
+ }
+ return writer.closeAndOpenReader();
+ }
+ catch (Throwable e)
+ {
+ writer.abort();
+ if (e instanceof IOException)
+ throw (IOException) e;
+ else
+ throw Throwables.propagate(e);
+ }
+ }
+
+ @Override
+ protected long totalSize()
+ {
+ long size = 0;
+ // calculate total length of transferring chunks
+ for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
+ size += chunk.length + 4; // 4 bytes for CRC
+ return size;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
new file mode 100644
index 0000000..80fcef5
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.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.streaming.compress;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamWriter;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamWriter for compressed SSTable.
+ */
+public class CompressedStreamWriter extends StreamWriter
+{
+ public static final int CHUNK_SIZE = 10 * 1024 * 1024;
+
+ private final CompressionInfo compressionInfo;
+
+ public CompressedStreamWriter(SSTableReader sstable, Collection<Pair<Long, Long>> sections, CompressionInfo compressionInfo, StreamSession session)
+ {
+ super(sstable, sections, session);
+ this.compressionInfo = compressionInfo;
+ }
+
+ @Override
+ public void write(WritableByteChannel channel) throws IOException
+ {
+ long totalSize = totalSize();
+ RandomAccessReader file = sstable.openDataReader();
+ FileChannel fc = file.getChannel();
+
+ long progress = 0L;
+ // calculate chunks to transfer. we want to send continuous chunks altogether.
+ List<Pair<Long, Long>> sections = getTransferSections(compressionInfo.chunks);
+ try
+ {
+ // stream each of the required sections of the file
+ for (Pair<Long, Long> section : sections)
+ {
+ // length of the section to stream
+ long length = section.right - section.left;
+ // tracks write progress
+ long bytesTransferred = 0;
+ while (bytesTransferred < length)
+ {
+ int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
+ limiter.acquire(toTransfer);
+ long lastWrite = fc.transferTo(section.left + bytesTransferred, toTransfer, channel);
+ bytesTransferred += lastWrite;
+ progress += lastWrite;
+ session.progress(sstable.descriptor, ProgressInfo.Direction.OUT, progress, totalSize);
+ }
+ }
+ }
+ finally
+ {
+ // no matter what happens close file
+ FileUtils.closeQuietly(file);
+ }
+
+ sstable.releaseReference();
+ }
+
+ @Override
+ protected long totalSize()
+ {
+ long size = 0;
+ // calculate total length of transferring chunks
+ for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
+ size += chunk.length + 4; // 4 bytes for CRC
+ return size;
+ }
+
+ // chunks are assumed to be sorted by offset
+ private List<Pair<Long, Long>> getTransferSections(CompressionMetadata.Chunk[] chunks)
+ {
+ List<Pair<Long, Long>> transferSections = new ArrayList<>();
+ Pair<Long, Long> lastSection = null;
+ for (CompressionMetadata.Chunk chunk : chunks)
+ {
+ if (lastSection != null)
+ {
+ if (chunk.offset == lastSection.right)
+ {
+ // extend previous section to end of this chunk
+ lastSection = Pair.create(lastSection.left, chunk.offset + chunk.length + 4); // 4 bytes for CRC
+ }
+ else
+ {
+ transferSections.add(lastSection);
+ lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
+ }
+ }
+ else
+ {
+ lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
+ }
+ }
+ if (lastSection != null)
+ transferSections.add(lastSection);
+ return transferSections;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
new file mode 100644
index 0000000..b561abc
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.IOException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.cassandra.streaming.StreamSession;
+
+public class CompleteMessage extends StreamMessage
+{
+ public static Serializer<CompleteMessage> serializer = new Serializer<CompleteMessage>()
+ {
+ public CompleteMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+ {
+ return new CompleteMessage();
+ }
+
+ public void serialize(CompleteMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException {}
+ };
+
+ public CompleteMessage()
+ {
+ super(Type.COMPLETE);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/FileMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessage.java b/src/java/org/apache/cassandra/streaming/messages/FileMessage.java
new file mode 100644
index 0000000..fe05eac
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessage.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.List;
+
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.streaming.StreamReader;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamWriter;
+import org.apache.cassandra.streaming.compress.CompressedStreamReader;
+import org.apache.cassandra.streaming.compress.CompressedStreamWriter;
+import org.apache.cassandra.streaming.compress.CompressionInfo;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * FileMessage is used to transfer/receive the part(or whole) of a SSTable data file.
+ */
+public class FileMessage extends StreamMessage
+{
+ public static Serializer<FileMessage> serializer = new Serializer<FileMessage>()
+ {
+ public FileMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+ {
+ DataInputStream input = new DataInputStream(Channels.newInputStream(in));
+ FileMessageHeader header = FileMessageHeader.serializer.deserialize(input, version);
+ StreamReader reader = header.compressionInfo == null ? new StreamReader(header, session)
+ : new CompressedStreamReader(header, session);
+
+ try
+ {
+ return new FileMessage(reader.read(in), header);
+ }
+ catch (Throwable e)
+ {
+ session.doRetry(header, e);
+ return null;
+ }
+ }
+
+ public void serialize(FileMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+ {
+ DataOutput output = new DataOutputStream(Channels.newOutputStream(out));
+ FileMessageHeader.serializer.serialize(message.header, output, version);
+ StreamWriter writer = message.header.compressionInfo == null ?
+ new StreamWriter(message.sstable, message.header.sections, session) :
+ new CompressedStreamWriter(message.sstable,
+ message.header.sections,
+ message.header.compressionInfo, session);
+ writer.write(out);
+ session.fileSent(message.header);
+ }
+ };
+
+ public final FileMessageHeader header;
+ public final SSTableReader sstable;
+
+ public FileMessage(SSTableReader sstable, FileMessageHeader header)
+ {
+ super(Type.FILE);
+ this.header = header;
+ this.sstable = sstable;
+ }
+
+ public FileMessage(SSTableReader sstable, int sequenceNumber, long estimatedKeys, List<Pair<Long, Long>> sections)
+ {
+ super(Type.FILE);
+ this.sstable = sstable;
+
+ CompressionInfo compressionInfo = null;
+ if (sstable.compression)
+ {
+ CompressionMetadata meta = sstable.getCompressionMetadata();
+ compressionInfo = new CompressionInfo(meta.getChunksForSections(sections), meta.parameters);
+ }
+ this.header = new FileMessageHeader(sstable.metadata.cfId,
+ sequenceNumber,
+ sstable.descriptor.version.toString(),
+ estimatedKeys,
+ sections,
+ compressionInfo);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
new file mode 100644
index 0000000..761e086
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -0,0 +1,151 @@
+/*
+ * 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.streaming.messages;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.compress.CompressionInfo;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+/**
+ * StreamingFileHeader is appended before sending actual data to describe what it's sending.
+ */
+public class FileMessageHeader
+{
+ public static IVersionedSerializer<FileMessageHeader> serializer = new FileMessageHeaderSerializer();
+
+ public final UUID cfId;
+ public final int sequenceNumber;
+ /** SSTable version */
+ public final String version;
+ public final long estimatedKeys;
+ public final List<Pair<Long, Long>> sections;
+ public final CompressionInfo compressionInfo;
+
+ public FileMessageHeader(UUID cfId,
+ int sequenceNumber,
+ String version,
+ long estimatedKeys,
+ List<Pair<Long, Long>> sections,
+ CompressionInfo compressionInfo)
+ {
+ this.cfId = cfId;
+ this.sequenceNumber = sequenceNumber;
+ this.version = version;
+ this.estimatedKeys = estimatedKeys;
+ this.sections = sections;
+ this.compressionInfo = compressionInfo;
+ }
+
+ /**
+ * @return total file size to transfer in bytes
+ */
+ public long size()
+ {
+ long size = 0;
+ if (compressionInfo != null)
+ {
+ // calculate total length of transferring chunks
+ for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
+ size += chunk.length + 4; // 4 bytes for CRC
+ }
+ else
+ {
+ for (Pair<Long, Long> section : sections)
+ size += section.right - section.left;
+ }
+ return size;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ FileMessageHeader that = (FileMessageHeader) o;
+ return sequenceNumber == that.sequenceNumber && cfId.equals(that.cfId);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = cfId.hashCode();
+ result = 31 * result + sequenceNumber;
+ return result;
+ }
+
+ static class FileMessageHeaderSerializer implements IVersionedSerializer<FileMessageHeader>
+ {
+ public void serialize(FileMessageHeader header, DataOutput out, int version) throws IOException
+ {
+ UUIDSerializer.serializer.serialize(header.cfId, out, version);
+ out.writeInt(header.sequenceNumber);
+ out.writeUTF(header.version);
+ out.writeLong(header.estimatedKeys);
+
+ out.writeInt(header.sections.size());
+ for (Pair<Long, Long> section : header.sections)
+ {
+ out.writeLong(section.left);
+ out.writeLong(section.right);
+ }
+ CompressionInfo.serializer.serialize(header.compressionInfo, out, version);
+ }
+
+ public FileMessageHeader deserialize(DataInput in, int version) throws IOException
+ {
+ UUID cfId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+ int sequenceNumber = in.readInt();
+ String sstableVersion = in.readUTF();
+ long estimatedKeys = in.readLong();
+ int count = in.readInt();
+ List<Pair<Long, Long>> sections = new ArrayList<>(count);
+ for (int k = 0; k < count; k++)
+ sections.add(Pair.create(in.readLong(), in.readLong()));
+ CompressionInfo compressionInfo = CompressionInfo.serializer.deserialize(in, MessagingService.current_version);
+ return new FileMessageHeader(cfId, sequenceNumber, sstableVersion, estimatedKeys, sections, compressionInfo);
+ }
+
+ public long serializedSize(FileMessageHeader header, int version)
+ {
+ long size = UUIDSerializer.serializer.serializedSize(header.cfId, version);
+ size += TypeSizes.NATIVE.sizeof(header.sequenceNumber);
+ size += TypeSizes.NATIVE.sizeof(header.version);
+ size += TypeSizes.NATIVE.sizeof(header.estimatedKeys);
+
+ size += TypeSizes.NATIVE.sizeof(header.sections.size());
+ for (Pair<Long, Long> section : header.sections)
+ {
+ size += TypeSizes.NATIVE.sizeof(section.left);
+ size += TypeSizes.NATIVE.sizeof(section.right);
+ }
+ size += CompressionInfo.serializer.serializedSize(header.compressionInfo, version);
+ return size;
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
new file mode 100644
index 0000000..ba94592
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
@@ -0,0 +1,91 @@
+/*
+ * 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.streaming.messages;
+
+import java.io.*;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.cassandra.streaming.StreamRequest;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+
+public class PrepareMessage extends StreamMessage
+{
+ public static Serializer<PrepareMessage> serializer = new Serializer<PrepareMessage>()
+ {
+ public PrepareMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+ {
+ DataInput input = new DataInputStream(Channels.newInputStream(in));
+ PrepareMessage message = new PrepareMessage();
+ // requests
+ int numRequests = input.readInt();
+ for (int i = 0; i < numRequests; i++)
+ message.requests.add(StreamRequest.serializer.deserialize(input, version));
+ // summaries
+ int numSummaries = input.readInt();
+ for (int i = 0; i < numSummaries; i++)
+ message.summaries.add(StreamSummary.serializer.deserialize(input, version));
+ return message;
+ }
+
+ public void serialize(PrepareMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+ {
+ DataOutput output = new DataOutputStream(Channels.newOutputStream(out));
+ // requests
+ output.writeInt(message.requests.size());
+ for (StreamRequest request : message.requests)
+ StreamRequest.serializer.serialize(request, output, version);
+ // summaries
+ output.writeInt(message.summaries.size());
+ for (StreamSummary summary : message.summaries)
+ StreamSummary.serializer.serialize(summary, output, version);
+ }
+ };
+
+ /**
+ * Streaming requests
+ */
+ public final Collection<StreamRequest> requests = new ArrayList<>();
+
+ /**
+ * Summaries of streaming out
+ */
+ public final Collection<StreamSummary> summaries = new ArrayList<>();
+
+ public PrepareMessage()
+ {
+ super(Type.PREPARE);
+ }
+
+ @Override
+ public String toString()
+ {
+ final StringBuilder sb = new StringBuilder("PrepareMessage{");
+ sb.append(requests.size()).append(" requests, ");
+ int totalFile = 0;
+ for (StreamSummary summary : summaries)
+ totalFile += summary.files;
+ sb.append(totalFile).append(" files receiving");
+ sb.append('}');
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java b/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
new file mode 100644
index 0000000..67b4dee
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.*;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.UUID;
+
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+public class RetryMessage extends StreamMessage
+{
+ public static Serializer<RetryMessage> serializer = new Serializer<RetryMessage>()
+ {
+ public RetryMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+ {
+ DataInput input = new DataInputStream(Channels.newInputStream(in));
+ return new RetryMessage(UUIDSerializer.serializer.deserialize(input, MessagingService.current_version), input.readInt());
+ }
+
+ public void serialize(RetryMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+ {
+ DataOutput output = new DataOutputStream(Channels.newOutputStream(out));
+ UUIDSerializer.serializer.serialize(message.cfId, output, MessagingService.current_version);
+ output.writeInt(message.sequenceNumber);
+ }
+ };
+
+ public final UUID cfId;
+ public final int sequenceNumber;
+
+ public RetryMessage(UUID cfId, int sequenceNumber)
+ {
+ super(Type.RETRY);
+ this.cfId = cfId;
+ this.sequenceNumber = sequenceNumber;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
new file mode 100644
index 0000000..450f67d
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.IOException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.cassandra.streaming.StreamSession;
+
+public class SessionFailedMessage extends StreamMessage
+{
+ public static Serializer<SessionFailedMessage> serializer = new Serializer<SessionFailedMessage>()
+ {
+ public SessionFailedMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+ {
+ return new SessionFailedMessage();
+ }
+
+ public void serialize(SessionFailedMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException {}
+ };
+
+ public SessionFailedMessage()
+ {
+ super(Type.SESSION_FAILED);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
new file mode 100644
index 0000000..d1e797c
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.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.streaming.messages;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+/**
+ * StreamInitMessage is first sent from the node where {@link org.apache.cassandra.streaming.StreamSession} is started,
+ * to initiate corresponding {@link org.apache.cassandra.streaming.StreamSession} on the other side.
+ */
+public class StreamInitMessage
+{
+ public static IVersionedSerializer<StreamInitMessage> serializer = new StreamInitMessageSerializer();
+
+ public final UUID planId;
+ public final String description;
+
+ public StreamInitMessage(UUID planId, String description)
+ {
+ this.planId = planId;
+ this.description = description;
+ }
+
+ /**
+ * Create serialized message.
+ *
+ * @param compress true if message is compressed
+ * @param version Streaming protocol version
+ * @return serialized message in ByteBuffer format
+ */
+ public ByteBuffer createMessage(boolean compress, int version)
+ {
+ int header = 0;
+ // set compression bit.
+ if (compress)
+ header |= 4;
+ // set streaming bit
+ header |= 8;
+ // Setting up the version bit
+ header |= (version << 8);
+
+ /* Adding the StreamHeader which contains the session Id along
+ * with the pendingfile info for the stream.
+ * | Session Id | Pending File Size | Pending File | Bool more files |
+ * | No. of Pending files | Pending Files ... |
+ */
+ byte[] bytes;
+ try
+ {
+ int size = (int)StreamInitMessage.serializer.serializedSize(this, version);
+ DataOutputBuffer buffer = new DataOutputBuffer(size);
+ StreamInitMessage.serializer.serialize(this, buffer, version);
+ bytes = buffer.getData();
+ }
+ catch (IOException e)
+ {
+ throw new RuntimeException(e);
+ }
+ assert bytes.length > 0;
+
+ ByteBuffer buffer = ByteBuffer.allocate(4 + 4 + bytes.length);
+ buffer.putInt(MessagingService.PROTOCOL_MAGIC);
+ buffer.putInt(header);
+ buffer.put(bytes);
+ buffer.flip();
+ return buffer;
+ }
+
+ private static class StreamInitMessageSerializer implements IVersionedSerializer<StreamInitMessage>
+ {
+ public void serialize(StreamInitMessage message, DataOutput out, int version) throws IOException
+ {
+ UUIDSerializer.serializer.serialize(message.planId, out, MessagingService.current_version);
+ out.writeUTF(message.description);
+ }
+
+ public StreamInitMessage deserialize(DataInput in, int version) throws IOException
+ {
+ UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+ return new StreamInitMessage(planId, in.readUTF());
+ }
+
+ public long serializedSize(StreamInitMessage message, int version)
+ {
+ long size = UUIDSerializer.serializer.serializedSize(message.planId, MessagingService.current_version);
+ size += TypeSizes.NATIVE.sizeof(message.description);
+ return size;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
new file mode 100644
index 0000000..f737675
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.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.streaming.messages;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.cassandra.streaming.StreamSession;
+
+/**
+ * StreamMessage is an abstract base class that every messages in streaming protocol inherit.
+ *
+ * Every message carries message type({@link Type}) and streaming protocol version byte.
+ */
+public abstract class StreamMessage
+{
+ /** Streaming protocol version */
+ public static final int CURRENT_VERSION = 1;
+
+ public static void serialize(StreamMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+ {
+ ByteBuffer buff = ByteBuffer.allocate(1);
+ // message type
+ buff.put(message.type.type);
+ buff.flip();
+ out.write(buff);
+ message.type.serializer.serialize(message, out, version, session);
+ }
+
+ public static StreamMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+ {
+ ByteBuffer buff = ByteBuffer.allocate(1);
+ in.read(buff);
+ buff.flip();
+ Type type = Type.get(buff.get());
+ return type.serializer.deserialize(in, version, session);
+ }
+
+ /** StreamMessage serializer */
+ public static interface Serializer<V extends StreamMessage>
+ {
+ V deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException;
+ void serialize(V message, WritableByteChannel out, int version, StreamSession session) throws IOException;
+ }
+
+ /** StreamMessage types */
+ public static enum Type
+ {
+ PREPARE(1, 5, PrepareMessage.serializer),
+ FILE(2, 0, FileMessage.serializer),
+ RETRY(3, 1, RetryMessage.serializer),
+ COMPLETE(4, 4, CompleteMessage.serializer),
+ SESSION_FAILED(5, 5, SessionFailedMessage.serializer);
+
+ public static Type get(byte type)
+ {
+ for (Type t : Type.values())
+ {
+ if (t.type == type)
+ return t;
+ }
+ throw new IllegalArgumentException("Unknown type " + type);
+ }
+
+ private final byte type;
+ public final int priority;
+ public final Serializer<StreamMessage> serializer;
+
+ @SuppressWarnings("unchecked")
+ private Type(int type, int priority, Serializer serializer)
+ {
+ this.type = (byte) type;
+ this.priority = priority;
+ this.serializer = serializer;
+ }
+ }
+
+ public final Type type;
+
+ protected StreamMessage(Type type)
+ {
+ this.type = type;
+ }
+
+ /**
+ * @return priority of this message. higher value, higher priority.
+ */
+ public int getPriority()
+ {
+ return type.priority;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/tools/BulkLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index d2f3c4d..12650a2 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -24,6 +24,11 @@ import java.util.*;
import java.util.concurrent.TimeUnit;
import org.apache.commons.cli.*;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.config.DatabaseDescriptor;
@@ -32,17 +37,10 @@ import org.apache.cassandra.db.Table;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.io.sstable.SSTableLoader;
-import org.apache.cassandra.streaming.PendingFile;
+import org.apache.cassandra.streaming.*;
import org.apache.cassandra.thrift.*;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.OutputHandler;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-
-import com.google.common.util.concurrent.Uninterruptibles;
public class BulkLoader
{
@@ -61,52 +59,20 @@ public class BulkLoader
public static void main(String args[])
{
LoaderOptions options = LoaderOptions.parseArgs(args);
+ OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
+ SSTableLoader loader = new SSTableLoader(options.directory, new ExternalClient(options.hosts, options.rpcPort, options.user, options.passwd), handler);
+ DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(options.throttle);
+ StreamResultFuture future = loader.stream(options.ignores);
+ future.addEventListener(new ProgressIndicator(handler));
try
{
- OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
- SSTableLoader loader = new SSTableLoader(options.directory, new ExternalClient(handler, options.hosts, options.rpcPort, options.user, options.passwd), handler);
- DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(options.throttle);
- SSTableLoader.LoaderFuture future = loader.stream(options.ignores);
-
- if (options.noProgress)
- {
- future.get();
- }
- else
- {
- ProgressIndicator indicator = new ProgressIndicator(future.getPendingFiles());
- indicator.start();
- System.out.println("");
- boolean printEnd = false;
- while (!future.isDone())
- {
- if (indicator.printProgress())
- {
- // We're done with streaming
- System.out.println("\nWaiting for targets to rebuild indexes ...");
- printEnd = true;
- future.get();
- assert future.isDone();
- }
- else
- {
- Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
- }
- }
- if (!printEnd)
- indicator.printProgress();
- if (future.hadFailures())
- {
- System.err.println("Streaming to the following hosts failed:");
- System.err.println(future.getFailedHosts());
- System.exit(1);
- }
- }
-
+ future.get();
System.exit(0); // We need that to stop non daemonized threads
}
catch (Exception e)
{
+ System.err.println("Streaming to the following hosts failed:");
+ System.err.println(loader.getFailedHosts());
System.err.println(e.getMessage());
if (options.debug)
e.printStackTrace(System.err);
@@ -115,62 +81,63 @@ public class BulkLoader
}
// Return true when everything is at 100%
- static class ProgressIndicator
+ static class ProgressIndicator implements StreamEventHandler
{
- private final Map<InetAddress, Collection<PendingFile>> filesByHost;
+ private final Set<ProgressInfo> progresses = new HashSet<>();
+ private final OutputHandler handler;
+
private long start;
private long lastProgress;
private long lastTime;
- public ProgressIndicator(Map<InetAddress, Collection<PendingFile>> filesByHost)
- {
- this.filesByHost = new HashMap<InetAddress, Collection<PendingFile>>(filesByHost);
- }
-
- public void start()
+ public ProgressIndicator(OutputHandler handler)
{
+ this.handler = handler;
start = lastTime = System.nanoTime();
}
- public boolean printProgress()
+ public void onSuccess(StreamState finalState) {}
+ public void onFailure(Throwable t) {}
+
+ public void handleStreamEvent(StreamEvent event)
{
- boolean done = true;
- StringBuilder sb = new StringBuilder();
- sb.append("\rprogress: ");
- long totalProgress = 0;
- long totalSize = 0;
- for (Map.Entry<InetAddress, Collection<PendingFile>> entry : filesByHost.entrySet())
+ if (event.eventType == StreamEvent.Type.FILE_PROGRESS)
{
- long progress = 0;
- long size = 0;
- int completed = 0;
- Collection<PendingFile> pendings = entry.getValue();
- for (PendingFile f : pendings)
+ ProgressInfo progressInfo = ((StreamEvent.ProgressEvent) event).progress;
+
+ // update progress
+ if (progresses.contains(progressInfo))
+ progresses.remove(progressInfo);
+ progresses.add(progressInfo);
+
+ StringBuilder sb = new StringBuilder();
+ sb.append("\rprogress: ");
+
+ long totalProgress = 0;
+ long totalSize = 0;
+ long completed = 0;
+ for (ProgressInfo entry : progresses)
{
- progress += f.progress;
- size += f.size;
- if (f.progress == f.size)
+ if (entry.currentBytes == entry.totalBytes)
completed++;
+ totalProgress += entry.currentBytes;
+ totalSize += entry.totalBytes;
+ sb.append("[").append(entry.peer);
+ sb.append(" ").append(completed).append("/").append(progresses.size());
+ sb.append(" (").append(entry.totalBytes == 0 ? 100L : entry.currentBytes * 100L / entry.totalBytes).append(")] ");
}
- totalProgress += progress;
- totalSize += size;
- if (completed != pendings.size())
- done = false;
- sb.append("[").append(entry.getKey());
- sb.append(" ").append(completed).append("/").append(pendings.size());
- sb.append(" (").append(size == 0 ? 100L : progress * 100L / size).append(")] ");
+ long time = System.nanoTime();
+ long deltaTime = TimeUnit.NANOSECONDS.toMillis(time - lastTime);
+ lastTime = time;
+ long deltaProgress = totalProgress - lastProgress;
+ lastProgress = totalProgress;
+
+ sb.append("[total: ").append(totalSize == 0 ? 100L : totalProgress * 100L / totalSize).append(" - ");
+ sb.append(mbPerSec(deltaProgress, deltaTime)).append("MB/s");
+ sb.append(" (avg: ").append(mbPerSec(totalProgress, TimeUnit.NANOSECONDS.toMillis(time - start))).append("MB/s)]");
+
+ handler.output(sb.toString());
}
- long time = System.nanoTime();
- long deltaTime = TimeUnit.NANOSECONDS.toMillis(time - lastTime);
- lastTime = time;
- long deltaProgress = totalProgress - lastProgress;
- lastProgress = totalProgress;
-
- sb.append("[total: ").append(totalSize == 0 ? 100L : totalProgress * 100L / totalSize).append(" - ");
- sb.append(mbPerSec(deltaProgress, deltaTime)).append("MB/s");
- sb.append(" (avg: ").append(mbPerSec(totalProgress, TimeUnit.NANOSECONDS.toMillis(time - start))).append("MB/s)]");
- System.out.print(sb.toString());
- return done;
}
private int mbPerSec(long bytes, long timeInMs)
@@ -182,13 +149,13 @@ public class BulkLoader
static class ExternalClient extends SSTableLoader.Client
{
- private final Set<String> knownCfs = new HashSet<String>();
+ private final Set<String> knownCfs = new HashSet<>();
private final Set<InetAddress> hosts;
private final int rpcPort;
private final String user;
private final String passwd;
- public ExternalClient(OutputHandler outputHandler, Set<InetAddress> hosts, int port, String user, String passwd)
+ public ExternalClient(Set<InetAddress> hosts, int port, String user, String passwd)
{
super();
this.hosts = hosts;
@@ -213,7 +180,7 @@ public class BulkLoader
for (TokenRange tr : client.describe_ring(keyspace))
{
- Range<Token> range = new Range<Token>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token));
+ Range<Token> range = new Range<>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token));
for (String ep : tr.endpoints)
{
addRangeForEndpoint(range, InetAddress.getByName(ep));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index b0b9e23..48ae998 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -45,6 +45,9 @@ import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
import org.apache.cassandra.net.MessagingServiceMBean;
import org.apache.cassandra.service.CacheServiceMBean;
import org.apache.cassandra.service.StorageProxyMBean;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.SessionInfo;
import org.apache.cassandra.utils.EstimatedHistogram;
import org.apache.cassandra.utils.Pair;
@@ -614,56 +617,34 @@ public class NodeCmd
public void printNetworkStats(final InetAddress addr, PrintStream outs)
{
outs.printf("Mode: %s%n", probe.getOperationMode());
- Set<InetAddress> hosts = addr == null ? probe.getStreamDestinations() : new HashSet<InetAddress>(){{add(addr);}};
- if (hosts.size() == 0)
+ Set<StreamState> statuses = probe.getStreamStatus();
+ if (statuses.isEmpty())
outs.println("Not sending any streams.");
- for (InetAddress host : hosts)
+ for (StreamState status : statuses)
{
- try
+ outs.printf("%s %s%n", status.description, status.planId.toString());
+ for (SessionInfo info : status.sessions)
{
- List<String> files = probe.getFilesDestinedFor(host);
- if (files.size() > 0)
+ outs.printf(" %s%n", info.peer.toString());
+ if (!info.receivingSummaries.isEmpty())
{
- outs.printf("Streaming to: %s%n", host);
- for (String file : files)
- outs.printf(" %s%n", file);
+ outs.printf(" Receiving %d files, %d bytes total%n", info.getTotalFilesToReceive(), info.getTotalSizeToReceive());
+ for (ProgressInfo progress : info.getReceivingFiles())
+ {
+ outs.printf(" %s%n", progress.toString());
+ }
}
- else
+ if (!info.sendingSummaries.isEmpty())
{
- outs.printf(" Nothing streaming to %s%n", host);
+ outs.printf(" Sending %d files, %d bytes total%n", info.getTotalFilesToSend(), info.getTotalSizeToSend());
+ for (ProgressInfo progress : info.getSendingFiles())
+ {
+ outs.printf(" %s%n", progress.toString());
+ }
}
}
- catch (IOException ex)
- {
- outs.printf(" Error retrieving file data for %s%n", host);
- }
}
- hosts = addr == null ? probe.getStreamSources() : new HashSet<InetAddress>(){{add(addr); }};
- if (hosts.size() == 0)
- outs.println("Not receiving any streams.");
- for (InetAddress host : hosts)
- {
- try
- {
- List<String> files = probe.getIncomingFiles(host);
- if (files.size() > 0)
- {
- outs.printf("Streaming from: %s%n", host);
- for (String file : files)
- outs.printf(" %s%n", file);
- }
- else
- {
- outs.printf(" Nothing streaming from %s%n", host);
- }
- }
- catch (IOException ex)
- {
- outs.printf(" Error retrieving file data for %s%n", host);
- }
- }
-
outs.printf("Read Repair Statistics:%nAttempted: %d%nMismatch (Blocking): %d%nMismatch (Background): %d%n", probe.getReadRepairAttempted(), probe.getReadRepairRepairedBlocking(), probe.getReadRepairRepairedBackground());
MessagingServiceMBean ms = probe.msProxy;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index af94d14..452e865 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -50,8 +50,8 @@ import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.net.MessagingServiceMBean;
import org.apache.cassandra.service.*;
-import org.apache.cassandra.streaming.StreamingService;
-import org.apache.cassandra.streaming.StreamingServiceMBean;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.streaming.StreamManagerMBean;
import org.apache.cassandra.utils.SimpleCondition;
/**
@@ -73,7 +73,7 @@ public class NodeProbe
private StorageServiceMBean ssProxy;
private MemoryMXBean memProxy;
private RuntimeMXBean runtimeProxy;
- private StreamingServiceMBean streamProxy;
+ private StreamManagerMBean streamProxy;
public MessagingServiceMBean msProxy;
private FailureDetectorMBean fdProxy;
private CacheServiceMBean cacheService;
@@ -150,8 +150,8 @@ public class NodeProbe
ssProxy = JMX.newMBeanProxy(mbeanServerConn, name, StorageServiceMBean.class);
name = new ObjectName(MessagingService.MBEAN_NAME);
msProxy = JMX.newMBeanProxy(mbeanServerConn, name, MessagingServiceMBean.class);
- name = new ObjectName(StreamingService.MBEAN_OBJECT_NAME);
- streamProxy = JMX.newMBeanProxy(mbeanServerConn, name, StreamingServiceMBean.class);
+ name = new ObjectName(StreamManagerMBean.OBJECT_NAME);
+ streamProxy = JMX.newMBeanProxy(mbeanServerConn, name, StreamManagerMBean.class);
name = new ObjectName(CompactionManager.MBEAN_OBJECT_NAME);
compactionProxy = JMX.newMBeanProxy(mbeanServerConn, name, CompactionManagerMBean.class);
name = new ObjectName(FailureDetector.MBEAN_NAME);
@@ -545,24 +545,9 @@ public class NodeProbe
return cfsProxy.getSSTablesForKey(key);
}
- public Set<InetAddress> getStreamDestinations()
+ public Set<StreamState> getStreamStatus()
{
- return streamProxy.getStreamDestinations();
- }
-
- public List<String> getFilesDestinedFor(InetAddress host) throws IOException
- {
- return streamProxy.getOutgoingFiles(host.getHostAddress());
- }
-
- public Set<InetAddress> getStreamSources()
- {
- return streamProxy.getStreamSources();
- }
-
- public List<String> getIncomingFiles(InetAddress host) throws IOException
- {
- return streamProxy.getIncomingFiles(host.getHostAddress());
+ return streamProxy.getCurrentStreams();
}
public String getOperationMode()
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index 7bbe72d..17fa3fd 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -36,7 +36,6 @@ import org.apache.cassandra.gms.IFailureDetectionEventListener;
import org.apache.cassandra.gms.IFailureDetector;
import org.apache.cassandra.locator.TokenMetadata;
import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.OperationType;
import static org.junit.Assert.*;
@@ -66,7 +65,7 @@ public class BootStrapperTest extends SchemaLoader
TokenMetadata tmd = ss.getTokenMetadata();
assertEquals(numOldNodes, tmd.sortedTokens().size());
- RangeStreamer s = new RangeStreamer(tmd, myEndpoint, OperationType.BOOTSTRAP);
+ RangeStreamer s = new RangeStreamer(tmd, myEndpoint, "Bootstrap");
IFailureDetector mockFailureDetector = new IFailureDetector()
{
public boolean isAlive(InetAddress ep)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/service/RemoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RemoveTest.java b/test/unit/org/apache/cassandra/service/RemoveTest.java
index 4f3217c..1b3eb48 100644
--- a/test/unit/org/apache/cassandra/service/RemoveTest.java
+++ b/test/unit/org/apache/cassandra/service/RemoveTest.java
@@ -27,29 +27,23 @@ import java.util.List;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.*;
import org.apache.cassandra.SchemaLoader;
import org.apache.cassandra.Util;
-import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.RandomPartitioner;
import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.gms.Gossiper;
import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.net.MessageIn;
import org.apache.cassandra.net.MessageOut;
import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.sink.IMessageSink;
import org.apache.cassandra.net.sink.SinkManager;
-import org.apache.cassandra.streaming.StreamUtil;
import org.apache.cassandra.utils.FBUtilities;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
public class RemoveTest
{
@@ -119,9 +113,6 @@ public class RemoveTest
@Test
public void testRemoveHostId() throws InterruptedException
{
- ReplicationSink rSink = new ReplicationSink();
- SinkManager.add(rSink);
-
// start removal in background and send replication confirmations
final AtomicBoolean success = new AtomicBoolean(false);
Thread remover = new Thread()
@@ -159,25 +150,4 @@ public class RemoveTest
assertTrue(success.get());
assertTrue(tmd.getLeavingEndpoints().isEmpty());
}
-
- /**
- * sink that captures STREAM_REQUEST messages and calls finishStreamRequest on it
- */
- class ReplicationSink implements IMessageSink
- {
- public MessageIn handleMessage(MessageIn msg, int id, InetAddress to)
- {
- if (!msg.verb.equals(MessagingService.Verb.STREAM_REQUEST))
- return msg;
-
- StreamUtil.finishStreamRequest(msg, to);
-
- return null;
- }
-
- public MessageOut handleMessage(MessageOut msg, int id, InetAddress to)
- {
- return msg;
- }
- }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/BootstrapTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/BootstrapTest.java b/test/unit/org/apache/cassandra/streaming/BootstrapTest.java
deleted file mode 100644
index 6578dde..0000000
--- a/test/unit/org/apache/cassandra/streaming/BootstrapTest.java
+++ /dev/null
@@ -1,54 +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.streaming;
-
-import static junit.framework.Assert.assertEquals;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.utils.Pair;
-
-import java.util.Arrays;
-
-import org.junit.Test;
-
-public class BootstrapTest extends SchemaLoader
-{
- @Test
- public void testGetNewNames() throws IOException
- {
- String ver = Descriptor.Version.current_version;
- Descriptor desc = Descriptor.fromFilename(new File("Keyspace1", "Keyspace1-Standard1-" + ver +"-500-Data.db").toString());
- // assert !desc.isLatestVersion; // minimum compatible version -- for now it is the latest as well
- PendingFile inContext = new PendingFile(null, desc, "Data.db", Arrays.asList(Pair.create(0L, 1L)), OperationType.BOOTSTRAP);
-
- PendingFile outContext = StreamIn.getContextMapping(inContext);
- // filename and generation are expected to have changed
- assert !inContext.getFilename().equals(outContext.getFilename());
-
- // nothing else should
- assertEquals(inContext.component, outContext.component);
- assertEquals(desc.ksname, outContext.desc.ksname);
- assertEquals(desc.cfname, outContext.desc.cfname);
- assertEquals(desc.version, outContext.desc.version);
- }
-}
[3/5] Streaming 2.0
Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamPlan.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java
new file mode 100644
index 0000000..78d50ad
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.*;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.utils.UUIDGen;
+
+/**
+ * {@link StreamPlan} is a helper class that builds StreamOperation of given configuration.
+ *
+ * This is the class you want to use for building streaming plan and starting streaming.
+ */
+public class StreamPlan
+{
+ private final UUID planId = UUIDGen.getTimeUUID();
+ private final String description;
+
+ // sessions per InetAddress of the other end.
+ private final Map<InetAddress, StreamSession> sessions = new HashMap<>();
+
+ private boolean flushBeforeTransfer = true;
+
+ /**
+ * Start building stream plan.
+ *
+ * @param description Stream type that describes this StreamPlan
+ */
+ public StreamPlan(String description)
+ {
+ this.description = description;
+ }
+
+ /**
+ * Request data in {@code keyspace} and {@code ranges} from specific node.
+ *
+ * @param from endpoint address to fetch data from.
+ * @param keyspace name of keyspace
+ * @param ranges ranges to fetch
+ * @return this object for chaining
+ */
+ public StreamPlan requestRanges(InetAddress from, String keyspace, Collection<Range<Token>> ranges)
+ {
+ return requestRanges(from, keyspace, ranges, new String[0]);
+ }
+
+ /**
+ * Request data in {@code columnFamilies} under {@code keyspace} and {@code ranges} from specific node.
+ *
+ * @param from endpoint address to fetch data from.
+ * @param keyspace name of keyspace
+ * @param ranges ranges to fetch
+ * @param columnFamilies specific column families
+ * @return this object for chaining
+ */
+ public StreamPlan requestRanges(InetAddress from, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+ {
+ StreamSession session = getOrCreateSession(from);
+ session.addStreamRequest(keyspace, ranges, Arrays.asList(columnFamilies));
+ return this;
+ }
+
+ /**
+ * Add transfer task to send data of specific keyspace and ranges.
+ *
+ * @param to endpoint address of receiver
+ * @param keyspace name of keyspace
+ * @param ranges ranges to send
+ * @return this object for chaining
+ */
+ public StreamPlan transferRanges(InetAddress to, String keyspace, Collection<Range<Token>> ranges)
+ {
+ return transferRanges(to, keyspace, ranges, new String[0]);
+ }
+
+ /**
+ * Add transfer task to send data of specific {@code columnFamilies} under {@code keyspace} and {@code ranges}.
+ *
+ * @param to endpoint address of receiver
+ * @param keyspace name of keyspace
+ * @param ranges ranges to send
+ * @param columnFamilies specific column families
+ * @return this object for chaining
+ */
+ public StreamPlan transferRanges(InetAddress to, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+ {
+ StreamSession session = getOrCreateSession(to);
+ session.addTransferRanges(keyspace, ranges, Arrays.asList(columnFamilies), flushBeforeTransfer);
+ return this;
+ }
+
+ /**
+ * Add transfer task to send given SSTable files.
+ *
+ * @param to endpoint address of receiver
+ * @param ranges ranges to send
+ * @param sstables files to send
+ * @return this object for chaining
+ */
+ public StreamPlan transferFiles(InetAddress to, Collection<Range<Token>> ranges, Collection<SSTableReader> sstables)
+ {
+ StreamSession session = getOrCreateSession(to);
+ session.addTransferFiles(ranges, sstables);
+ return this;
+ }
+
+ /**
+ * @return true if this plan has no plan to execute
+ */
+ public boolean isEmpty()
+ {
+ return sessions.isEmpty();
+ }
+
+ /**
+ * Execute this {@link StreamPlan} asynchronously.
+ *
+ * @return Future {@link StreamState} that you can use to listen on progress of streaming.
+ */
+ public StreamResultFuture execute()
+ {
+ return StreamResultFuture.startStreamingAsync(planId, description, sessions.values());
+ }
+
+ /**
+ * Set flushBeforeTransfer option.
+ * When it's true, will flush before streaming ranges. (Default: true)
+ *
+ * @param flushBeforeTransfer set to true when the node should flush before transfer
+ * @return this object for chaining
+ */
+ public StreamPlan flushBeforeTransfer(boolean flushBeforeTransfer)
+ {
+ this.flushBeforeTransfer = flushBeforeTransfer;
+ return this;
+ }
+
+ private StreamSession getOrCreateSession(InetAddress peer)
+ {
+ StreamSession session = sessions.get(peer);
+ if (session == null)
+ {
+ session = new StreamSession(peer);
+ sessions.put(peer, session);
+ }
+ return session;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
new file mode 100644
index 0000000..a01de3c
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -0,0 +1,120 @@
+/*
+ * 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.streaming;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Collection;
+import java.util.UUID;
+
+import com.google.common.base.Throwables;
+import com.ning.compress.lzf.LZFInputStream;
+
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.streaming.messages.FileMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamReader reads from stream and writes to SSTable.
+ */
+public class StreamReader
+{
+ protected final UUID cfId;
+ protected final long estimatedKeys;
+ protected final Collection<Pair<Long, Long>> sections;
+ protected final StreamSession session;
+
+ protected Descriptor desc;
+
+ public StreamReader(FileMessageHeader header, StreamSession session)
+ {
+ this.session = session;
+ this.cfId = header.cfId;
+ this.estimatedKeys = header.estimatedKeys;
+ this.sections = header.sections;
+ }
+
+ /**
+ * @param channel where this reads data from
+ * @return SSTable transferred
+ * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
+ */
+ public SSTableReader read(ReadableByteChannel channel) throws IOException
+ {
+ long totalSize = totalSize();
+
+ Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+ Directories.DataDirectory localDir = cfs.directories.getLocationCapableOfSize(totalSize);
+ if (localDir == null)
+ throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
+ desc = Descriptor.fromFilename(cfs.getTempSSTablePath(cfs.directories.getLocationForDisk(localDir)));
+
+ SSTableWriter writer = new SSTableWriter(desc.filenameFor(Component.DATA), estimatedKeys);
+ try
+ {
+ DataInputStream dis = new DataInputStream(new LZFInputStream(Channels.newInputStream(channel)));
+ BytesReadTracker in = new BytesReadTracker(dis);
+
+ while (in.getBytesRead() < totalSize)
+ {
+ writeRow(writer, in, cfs);
+ // TODO move this to BytesReadTracker
+ session.progress(desc, ProgressInfo.Direction.IN, in.getBytesRead(), totalSize);
+ }
+ return writer.closeAndOpenReader();
+ }
+ catch (Throwable e)
+ {
+ writer.abort();
+ if (e instanceof IOException)
+ throw (IOException) e;
+ else
+ throw Throwables.propagate(e);
+ }
+ }
+
+ protected long totalSize()
+ {
+ long size = 0;
+ for (Pair<Long, Long> section : sections)
+ size += section.right - section.left;
+ return size;
+ }
+
+ protected void writeRow(SSTableWriter writer, DataInput in, ColumnFamilyStore cfs) throws IOException
+ {
+ DecoratedKey key = StorageService.getPartitioner().decorateKey(ByteBufferUtil.readWithShortLength(in));
+ writer.appendFromStream(key, cfs.metadata, in);
+ cfs.invalidateCachedRow(key);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
new file mode 100644
index 0000000..af8c138
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Task that manages receiving files for the session for certain ColumnFamily.
+ */
+public class StreamReceiveTask extends StreamTask
+{
+ // number of files to receive
+ private final int totalFiles;
+ // total size of files to receive
+ private final long totalSize;
+
+ // holds references to SSTables received
+ protected Collection<SSTableReader> sstables;
+
+ public StreamReceiveTask(StreamSession session, UUID cfId, int totalFiles, long totalSize)
+ {
+ super(session, cfId);
+ this.totalFiles = totalFiles;
+ this.totalSize = totalSize;
+ this.sstables = new ArrayList<>(totalFiles);
+ }
+
+ /**
+ * Process received file.
+ *
+ * @param sstable SSTable file received.
+ */
+ public void receive(SSTableReader sstable)
+ {
+ assert cfId.equals(sstable.metadata.cfId);
+
+ sstables.add(sstable);
+ if (sstables.size() == totalFiles)
+ complete();
+ }
+
+ public int getTotalNumberOfFiles()
+ {
+ return totalFiles;
+ }
+
+ public long getTotalSize()
+ {
+ return totalSize;
+ }
+
+ // TODO should be run in background so that this does not block streaming
+ private void complete()
+ {
+ if (!SSTableReader.acquireReferences(sstables))
+ throw new AssertionError("We shouldn't fail acquiring a reference on a sstable that has just been transferred");
+ try
+ {
+ Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+ // add sstables and build secondary indexes
+ cfs.addSSTables(sstables);
+ cfs.indexManager.maybeBuildSecondaryIndexes(sstables, cfs.indexManager.allIndexesNames());
+ }
+ finally
+ {
+ SSTableReader.releaseReferences(sstables);
+ }
+
+ session.taskCompleted(this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamReply.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReply.java b/src/java/org/apache/cassandra/streaming/StreamReply.java
deleted file mode 100644
index d54e9cc..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamReply.java
+++ /dev/null
@@ -1,91 +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.streaming;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.UUID;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDSerializer;
-
-public class StreamReply
-{
- static enum Status
- {
- FILE_FINISHED,
- FILE_RETRY,
- SESSION_FINISHED,
- SESSION_FAILURE,
- }
-
- public static final IVersionedSerializer<StreamReply> serializer = new FileStatusSerializer();
-
- public final UUID sessionId;
- public final String file;
- public final Status action;
-
- public StreamReply(String file, UUID sessionId, Status action)
- {
- this.file = file;
- this.action = action;
- this.sessionId = sessionId;
- }
-
- public MessageOut<StreamReply> createMessage()
- {
- return new MessageOut<StreamReply>(MessagingService.Verb.STREAM_REPLY, this, serializer);
- }
-
- @Override
- public String toString()
- {
- return "StreamReply(" +
- "sessionId=" + sessionId +
- ", file='" + file + '\'' +
- ", action=" + action +
- ')';
- }
-
- private static class FileStatusSerializer implements IVersionedSerializer<StreamReply>
- {
- public void serialize(StreamReply reply, DataOutput out, int version) throws IOException
- {
- UUIDSerializer.serializer.serialize(reply.sessionId, out, MessagingService.current_version);
- out.writeUTF(reply.file);
- out.writeInt(reply.action.ordinal());
- }
-
- public StreamReply deserialize(DataInput in, int version) throws IOException
- {
- UUID sessionId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
- String targetFile = in.readUTF();
- Status action = Status.values()[in.readInt()];
- return new StreamReply(targetFile, sessionId, action);
- }
-
- public long serializedSize(StreamReply reply, int version)
- {
- return TypeSizes.NATIVE.sizeof(reply.sessionId) + TypeSizes.NATIVE.sizeof(reply.file) + TypeSizes.NATIVE.sizeof(reply.action.ordinal());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java b/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java
deleted file mode 100644
index b69b6d0..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java
+++ /dev/null
@@ -1,63 +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.streaming;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-
-public class StreamReplyVerbHandler implements IVerbHandler<StreamReply>
-{
- private static final Logger logger = LoggerFactory.getLogger(StreamReplyVerbHandler.class);
-
- public void doVerb(MessageIn<StreamReply> message, int id)
- {
- StreamReply reply = message.payload;
- logger.debug("Received StreamReply {}", reply);
- StreamOutSession session = StreamOutSession.get(reply.sessionId);
- if (session == null)
- {
- logger.debug("Received stream action " + reply.action + " for an unknown session from " + message.from);
- return;
- }
-
- switch (reply.action)
- {
- case FILE_FINISHED:
- logger.info("Successfully sent {} to {}", reply.file, message.from);
- session.validateCurrentFile(reply.file);
- session.startNext();
- break;
- case FILE_RETRY:
- session.validateCurrentFile(reply.file);
- logger.info("Need to re-stream file {} to {}", reply.file, message.from);
- session.retry();
- break;
- case SESSION_FINISHED:
- session.close(true);
- break;
- case SESSION_FAILURE:
- session.close(false);
- break;
- default:
- throw new RuntimeException("Cannot handle FileStatus.Action: " + reply.action);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamRequest.java b/src/java/org/apache/cassandra/streaming/StreamRequest.java
index 465d8bf..9d3fdb2 100644
--- a/src/java/org/apache/cassandra/streaming/StreamRequest.java
+++ b/src/java/org/apache/cassandra/streaming/StreamRequest.java
@@ -20,173 +20,77 @@ package org.apache.cassandra.streaming;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.HashSet;
import java.util.List;
-import java.util.UUID;
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.dht.AbstractBounds;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.CompactEndpointSerializationHelper;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDSerializer;
-/**
-* This class encapsulates the message that needs to be sent to nodes
-* that handoff data. The message contains information about ranges
-* that need to be transferred and the target node.
-*
-* If a file is specified, ranges and table will not. vice-versa should hold as well.
-*/
public class StreamRequest
{
public static final IVersionedSerializer<StreamRequest> serializer = new StreamRequestSerializer();
- protected final UUID sessionId;
- protected final InetAddress target;
-
- // if this is specified, ranges and table should not be.
- protected final PendingFile file;
+ public final String keyspace;
+ public final Collection<Range<Token>> ranges;
+ public final Collection<String> columnFamilies = new HashSet<>();
- // if these are specified, file should not be.
- protected final Collection<Range<Token>> ranges;
- protected final String table;
- protected final Iterable<ColumnFamilyStore> columnFamilies;
- protected final OperationType type;
-
- StreamRequest(InetAddress target, Collection<Range<Token>> ranges, String table, Iterable<ColumnFamilyStore> columnFamilies, UUID sessionId, OperationType type)
+ public StreamRequest(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies)
{
- this.target = target;
+ this.keyspace = keyspace;
this.ranges = ranges;
- this.table = table;
- this.columnFamilies = columnFamilies;
- this.sessionId = sessionId;
- this.type = type;
- file = null;
- }
-
- StreamRequest(InetAddress target, PendingFile file, UUID sessionId)
- {
- this.target = target;
- this.file = file;
- this.sessionId = sessionId;
- this.type = file.type;
- ranges = null;
- table = null;
- columnFamilies = null;
- }
-
- public MessageOut<StreamRequest> createMessage()
- {
- return new MessageOut<StreamRequest>(MessagingService.Verb.STREAM_REQUEST, this, serializer);
+ this.columnFamilies.addAll(columnFamilies);
}
- public String toString()
+ public static class StreamRequestSerializer implements IVersionedSerializer<StreamRequest>
{
- StringBuilder sb = new StringBuilder("");
- if (file == null)
+ public void serialize(StreamRequest request, DataOutput out, int version) throws IOException
{
- sb.append(table);
- sb.append("@");
- sb.append(columnFamilies.toString());
- sb.append("@");
- sb.append(target);
- sb.append("------->");
- for (Range<Token> range : ranges)
+ out.writeUTF(request.keyspace);
+ out.writeInt(request.ranges.size());
+ for (Range<Token> range : request.ranges)
{
- sb.append(range);
- sb.append(" ");
+ Token.serializer.serialize(range.left, out);
+ Token.serializer.serialize(range.right, out);
}
- sb.append(type);
- }
- else
- {
- sb.append(file.toString());
+ out.writeInt(request.columnFamilies.size());
+ for (String cf : request.columnFamilies)
+ out.writeUTF(cf);
}
- return sb.toString();
- }
- private static class StreamRequestSerializer implements IVersionedSerializer<StreamRequest>
- {
- public void serialize(StreamRequest srm, DataOutput out, int version) throws IOException
+ public StreamRequest deserialize(DataInput in, int version) throws IOException
{
- UUIDSerializer.serializer.serialize(srm.sessionId, out, MessagingService.current_version);
- CompactEndpointSerializationHelper.serialize(srm.target, out);
- if (srm.file != null)
+ String keyspace = in.readUTF();
+ int rangeCount = in.readInt();
+ List<Range<Token>> ranges = new ArrayList<>(rangeCount);
+ for (int i = 0; i < rangeCount; i++)
{
- out.writeBoolean(true);
- PendingFile.serializer.serialize(srm.file, out, version);
- }
- else
- {
- out.writeBoolean(false);
- out.writeUTF(srm.table);
- out.writeInt(srm.ranges.size());
- for (Range<Token> range : srm.ranges)
- AbstractBounds.serializer.serialize(range, out, version);
-
- out.writeUTF(srm.type.name());
-
- out.writeInt(Iterables.size(srm.columnFamilies));
- for (ColumnFamilyStore cfs : srm.columnFamilies)
- ColumnFamily.serializer.serializeCfId(cfs.metadata.cfId, out, version);
+ Token left = Token.serializer.deserialize(in);
+ Token right = Token.serializer.deserialize(in);
+ ranges.add(new Range<>(left, right));
}
+ int cfCount = in.readInt();
+ List<String> columnFamilies = new ArrayList<>(cfCount);
+ for (int i = 0; i < cfCount; i++)
+ columnFamilies.add(in.readUTF());
+ return new StreamRequest(keyspace, ranges, columnFamilies);
}
- public StreamRequest deserialize(DataInput in, int version) throws IOException
+ public long serializedSize(StreamRequest request, int version)
{
- UUID sessionId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
- InetAddress target = CompactEndpointSerializationHelper.deserialize(in);
- boolean singleFile = in.readBoolean();
- if (singleFile)
+ int size = TypeSizes.NATIVE.sizeof(request.keyspace);
+ size += TypeSizes.NATIVE.sizeof(request.ranges.size());
+ for (Range<Token> range : request.ranges)
{
- PendingFile file = PendingFile.serializer.deserialize(in, version);
- return new StreamRequest(target, file, sessionId);
+ size += Token.serializer.serializedSize(range.left, TypeSizes.NATIVE);
+ size += Token.serializer.serializedSize(range.right, TypeSizes.NATIVE);
}
- else
- {
- String table = in.readUTF();
- int size = in.readInt();
- List<Range<Token>> ranges = (size == 0) ? null : new ArrayList<Range<Token>>(size);
- for (int i = 0; i < size; ++i)
- ranges.add((Range<Token>) AbstractBounds.serializer.deserialize(in, version).toTokenBounds());
- OperationType type = OperationType.valueOf(in.readUTF());
-
- List<ColumnFamilyStore> stores = new ArrayList<ColumnFamilyStore>();
- int cfsSize = in.readInt();
- for (int i = 0; i < cfsSize; ++i)
- stores.add(Table.open(table).getColumnFamilyStore(ColumnFamily.serializer.deserializeCfId(in, version)));
-
- return new StreamRequest(target, ranges, table, stores, sessionId, type);
- }
- }
-
- public long serializedSize(StreamRequest sr, int version)
- {
- long size = TypeSizes.NATIVE.sizeof(sr.sessionId);
- size += CompactEndpointSerializationHelper.serializedSize(sr.target);
- size += TypeSizes.NATIVE.sizeof(true);
- if (sr.file != null)
- return size + PendingFile.serializer.serializedSize(sr.file, version);
-
- size += TypeSizes.NATIVE.sizeof(sr.table);
- size += TypeSizes.NATIVE.sizeof(sr.ranges.size());
- for (Range<Token> range : sr.ranges)
- size += AbstractBounds.serializer.serializedSize(range, version);
- size += TypeSizes.NATIVE.sizeof(sr.type.name());
- size += TypeSizes.NATIVE.sizeof(Iterables.size(sr.columnFamilies));
- for (ColumnFamilyStore cfs : sr.columnFamilies)
- size += ColumnFamily.serializer.cfIdSerializedSize(cfs.metadata.cfId, TypeSizes.NATIVE, version);
+ size += TypeSizes.NATIVE.sizeof(request.columnFamilies.size());
+ for (String cf : request.columnFamilies)
+ size += TypeSizes.NATIVE.sizeof(cf);
return size;
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java b/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java
deleted file mode 100644
index bbed34d..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java
+++ /dev/null
@@ -1,47 +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.streaming;
-
- import org.slf4j.Logger;
- import org.slf4j.LoggerFactory;
-
- import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-
-/**
- * This verb handler handles the StreamRequestMessage that is sent by
- * the node requesting range transfer.
-*/
-public class StreamRequestVerbHandler implements IVerbHandler<StreamRequest>
-{
- private static final Logger logger = LoggerFactory.getLogger(StreamRequestVerbHandler.class);
-
- public void doVerb(MessageIn<StreamRequest> message, int id)
- {
- if (logger.isDebugEnabled())
- logger.debug("Received a StreamRequestMessage from {}", message.from);
-
- StreamRequest srm = message.payload;
- if (logger.isDebugEnabled())
- logger.debug(srm.toString());
-
- StreamOutSession session = StreamOutSession.create(srm.table, message.from, srm.sessionId);
- StreamOut.transferRanges(session, srm.columnFamilies, srm.ranges, srm.type);
- }
-}
-
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
new file mode 100644
index 0000000..84332bd
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.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.streaming;
+
+import java.net.InetAddress;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
+
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * StreamResultFuture asynchronously returns the final {@link StreamState} of execution of {@link StreamPlan}.
+ * <p>
+ * You can attach {@link StreamEventHandler} to this object to listen on {@link StreamEvent}s to track progress of the streaming.
+ */
+public final class StreamResultFuture extends AbstractFuture<StreamState>
+{
+ // Executor that establish the streaming connection. Once we're connected to the other end, the rest of the streaming
+ // is directly handled by the ConnectionHandler incoming and outgoing threads.
+ private static final DebuggableThreadPoolExecutor streamExecutor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("StreamConnectionEstablisher",
+ FBUtilities.getAvailableProcessors());
+
+ public final UUID planId;
+ public final String description;
+ private final List<StreamEventHandler> eventListeners = Collections.synchronizedList(new ArrayList<StreamEventHandler>());
+ private final Set<UUID> ongoingSessions;
+ private final Map<InetAddress, SessionInfo> sessionStates = new NonBlockingHashMap<>();
+
+ /**
+ * Create new StreamResult of given {@code planId} and type.
+ *
+ * Constructor is package private. You need to use {@link StreamPlan#execute()} to get the instance.
+ *
+ * @param planId Stream plan ID
+ * @param description Stream description
+ * @param numberOfSessions number of sessions to wait for complete
+ */
+ private StreamResultFuture(UUID planId, String description, Set<UUID> sessions)
+ {
+ this.planId = planId;
+ this.description = description;
+ this.ongoingSessions = sessions;
+
+ // if there is no session to listen to, we immediately set result for returning
+ if (sessions.isEmpty())
+ set(getCurrentState());
+ }
+
+ static StreamResultFuture startStreamingAsync(UUID planId, String description, Collection<StreamSession> sessions)
+ {
+ Set<UUID> sessionsIds = new HashSet<>(sessions.size());
+ for (StreamSession session : sessions)
+ sessionsIds.add(session.id);
+
+ StreamResultFuture future = new StreamResultFuture(planId, description, sessionsIds);
+
+ StreamManager.instance.register(future);
+
+ // start sessions
+ for (StreamSession session : sessions)
+ {
+ session.register(future);
+ // register to gossiper/FD to fail on node failure
+ Gossiper.instance.register(session);
+ FailureDetector.instance.registerFailureDetectionEventListener(session);
+ streamExecutor.submit(session);
+ }
+ return future;
+ }
+
+ public void addEventListener(StreamEventHandler listener)
+ {
+ Futures.addCallback(this, listener);
+ eventListeners.add(listener);
+ }
+
+ /**
+ * @return Current snapshot of streaming progress.
+ */
+ public StreamState getCurrentState()
+ {
+ return new StreamState(planId, description, ImmutableSet.copyOf(sessionStates.values()));
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ StreamResultFuture that = (StreamResultFuture) o;
+ return planId.equals(that.planId);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return planId.hashCode();
+ }
+
+ void handleSessionPrepared(StreamSession session)
+ {
+ SessionInfo sessionInfo = session.getSessionInfo();
+ StreamEvent.SessionPreparedEvent event = new StreamEvent.SessionPreparedEvent(planId, sessionInfo);
+ sessionStates.put(sessionInfo.peer, sessionInfo);
+ fireStreamEvent(event);
+ }
+
+ void handleSessionComplete(StreamSession session)
+ {
+ Gossiper.instance.unregister(session);
+ FailureDetector.instance.unregisterFailureDetectionEventListener(session);
+
+ SessionInfo sessionInfo = session.getSessionInfo();
+ sessionStates.put(sessionInfo.peer, sessionInfo);
+ fireStreamEvent(new StreamEvent.SessionCompleteEvent(session));
+ maybeComplete(session.id);
+ }
+
+ public void handleProgress(ProgressInfo progress)
+ {
+ sessionStates.get(progress.peer).updateProgress(progress);
+ fireStreamEvent(new StreamEvent.ProgressEvent(planId, progress));
+ }
+
+ void fireStreamEvent(StreamEvent event)
+ {
+ // delegate to listener
+ for (StreamEventHandler listener : eventListeners)
+ listener.handleStreamEvent(event);
+ }
+
+ private synchronized void maybeComplete(UUID sessionId)
+ {
+ ongoingSessions.remove(sessionId);
+ if (ongoingSessions.isEmpty())
+ {
+ StreamState finalState = getCurrentState();
+ if (finalState.hasFailedSession())
+ setException(new StreamException(finalState, "Stream failed"));
+ else
+ set(finalState);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
new file mode 100644
index 0000000..7d96f43
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -0,0 +1,591 @@
+/*
+ * 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.streaming;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.*;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.gms.*;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.metrics.StreamingMetrics;
+import org.apache.cassandra.streaming.messages.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDGen;
+
+/**
+ * StreamSession is the center of Cassandra Streaming API.
+ *
+ * StreamSession on the both endpoints exchange messages and files until complete.
+ *
+ * It is created through {@link StreamPlan} on the initiator node,
+ * and also is created directly from connected socket on the other end when received init message.
+ *
+ * <p>
+ * StreamSession goes through several stages:
+ * <ol>
+ * <li>
+ * Init
+ * <p>StreamSession in one end send init message to the other end.</p>
+ * </li>
+ * <li>
+ * Prepare
+ * <p>StreamSession in both endpoints are created, so in this phase, they exchange
+ * request and summary messages to prepare receiving/streaming files in next phase.</p>
+ * </li>
+ * <li>
+ * Stream
+ * <p>StreamSessions in both ends stream and receive files.</p>
+ * </li>
+ * <li>
+ * Complete
+ * <p>Session completes if both endpoints completed by exchanging complete message.</p>
+ * </li>
+ * </ol>
+ */
+public class StreamSession implements Runnable, IEndpointStateChangeSubscriber, IFailureDetectionEventListener
+{
+ private static final Logger logger = LoggerFactory.getLogger(StreamSession.class);
+
+ public final UUID id = UUIDGen.getTimeUUID();
+ public final InetAddress peer;
+
+ // should not be null when session is started
+ private StreamResultFuture streamResult;
+
+ // stream requests to send to the peer
+ private final List<StreamRequest> requests = new ArrayList<>();
+ // streaming tasks are created and managed per ColumnFamily ID
+ private final Map<UUID, StreamTransferTask> transfers = new HashMap<>();
+ // data receivers, filled after receiving prepare message
+ private final Map<UUID, StreamReceiveTask> receivers = new HashMap<>();
+ private final StreamingMetrics metrics;
+
+ public final ConnectionHandler handler;
+
+ private int retries;
+
+ public static enum State
+ {
+ INITIALIZING,
+ PREPARING,
+ STREAMING,
+ WAIT_COMPLETE,
+ COMPLETE,
+ FAILED,
+ }
+
+ private volatile State state = State.INITIALIZING;
+
+ /**
+ * Create new streaming session with the peer.
+ *
+ * @param peer Address of streaming peer
+ */
+ public StreamSession(InetAddress peer)
+ {
+ this.peer = peer;
+ this.handler = new ConnectionHandler(this);
+ this.metrics = StreamingMetrics.get(peer);
+ }
+
+ /**
+ * Create streaming session from established connection.
+ *
+ * @param socket established connection
+ * @param protocolVersion Streaming protocol verison
+ */
+ public StreamSession(Socket socket, int protocolVersion)
+ {
+ this.peer = socket.getInetAddress();
+ this.handler = new ConnectionHandler(this, socket, protocolVersion);
+ this.metrics = StreamingMetrics.get(peer);
+ }
+
+ public UUID planId()
+ {
+ return streamResult == null ? null : streamResult.planId;
+ }
+
+ public String description()
+ {
+ return streamResult == null ? null : streamResult.description;
+ }
+
+ public static StreamSession startReceivingStreamAsync(UUID planId, String description, Socket socket, int version)
+ {
+ StreamSession session = new StreamSession(socket, version);
+ StreamResultFuture.startStreamingAsync(planId, description, Collections.singleton(session));
+ return session;
+ }
+
+ /**
+ * Bind this session to report to specific {@link StreamResultFuture}.
+ *
+ * @param streamResult result to report to
+ * @return this object for chaining
+ */
+ public StreamSession register(StreamResultFuture streamResult)
+ {
+ this.streamResult = streamResult;
+ return this;
+ }
+
+ /**
+ * Request data fetch task to this session.
+ *
+ * @param keyspace Requesting keyspace
+ * @param ranges Ranges to retrieve data
+ * @param columnFamilies ColumnFamily names. Can be empty if requesting all CF under the keyspace.
+ */
+ public void addStreamRequest(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies)
+ {
+ requests.add(new StreamRequest(keyspace, ranges, columnFamilies));
+ }
+
+ /**
+ * Set up transfer for specific keyspace/ranges/CFs
+ *
+ * @param keyspace Transfer keyspace
+ * @param ranges Transfer ranges
+ * @param columnFamilies Transfer ColumnFamilies
+ */
+ public void addTransferRanges(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies, boolean flushTables)
+ {
+ Collection<ColumnFamilyStore> stores = new HashSet<>();
+ // if columnfamilies are not specified, we add all cf under the keyspace
+ if (columnFamilies.isEmpty())
+ {
+ stores.addAll(Table.open(keyspace).getColumnFamilyStores());
+ }
+ else
+ {
+ for (String cf : columnFamilies)
+ stores.add(Table.open(keyspace).getColumnFamilyStore(cf));
+ }
+
+ if (flushTables)
+ flushSSTables(stores);
+
+ List<SSTableReader> sstables = Lists.newLinkedList();
+ for (ColumnFamilyStore cfStore : stores)
+ {
+ List<AbstractBounds<RowPosition>> rowBoundsList = Lists.newLinkedList();
+ for (Range<Token> range : ranges)
+ rowBoundsList.add(range.toRowBounds());
+ ColumnFamilyStore.ViewFragment view = cfStore.markReferenced(rowBoundsList);
+ sstables.addAll(view.sstables);
+ }
+ addTransferFiles(ranges, sstables);
+ }
+
+ /**
+ * Set up transfer of the specific SSTables.
+ * {@code sstables} must be marked as referenced so that not get deleted until transfer completes.
+ *
+ * @param ranges Transfer ranges
+ * @param sstables Transfer files
+ */
+ public void addTransferFiles(Collection<Range<Token>> ranges, Collection<SSTableReader> sstables)
+ {
+ for (SSTableReader sstable : sstables)
+ {
+ List<Pair<Long, Long>> sections = sstable.getPositionsForRanges(ranges);
+ if (sections.isEmpty())
+ {
+ // A reference was acquired on the sstable and we won't stream it
+ sstable.releaseReference();
+ continue;
+ }
+ long estimatedKeys = sstable.estimatedKeysForRanges(ranges);
+ UUID cfId = sstable.metadata.cfId;
+ StreamTransferTask task = transfers.get(cfId);
+ if (task == null)
+ {
+ task = new StreamTransferTask(this, cfId);
+ transfers.put(cfId, task);
+ }
+ task.addTransferFile(sstable, estimatedKeys, sections);
+ }
+ }
+
+ /**
+ * Start this stream session.
+ */
+ public void run()
+ {
+ assert streamResult != null : "No result is associated with this session";
+
+ try
+ {
+ if (handler.isConnected())
+ {
+ // if this session is created from remote...
+ handler.start();
+ }
+ else
+ {
+ if (requests.isEmpty() && transfers.isEmpty())
+ {
+ logger.debug("Session does not have any tasks.");
+ state(State.COMPLETE);
+ streamResult.handleSessionComplete(this);
+ }
+ else
+ {
+ handler.connect();
+ }
+ }
+ }
+ catch (IOException e)
+ {
+ onError(e);
+ }
+ }
+
+ /**
+ * Set current state to {@code newState}.
+ *
+ * @param newState new state to set
+ */
+ public void state(State newState)
+ {
+ state = newState;
+ }
+
+ /**
+ * @return current state
+ */
+ public State state()
+ {
+ return state;
+ }
+
+ /**
+ * Return if this session completed successfully.
+ *
+ * @return true if session completed successfully.
+ */
+ public boolean isSuccess()
+ {
+ return state == State.COMPLETE;
+ }
+
+ public void messageReceived(StreamMessage message)
+ {
+ switch (message.type)
+ {
+ case PREPARE:
+ PrepareMessage msg = (PrepareMessage) message;
+ prepare(msg.requests, msg.summaries);
+ break;
+
+ case FILE:
+ receive((FileMessage) message);
+ break;
+
+ case RETRY:
+ RetryMessage retry = (RetryMessage) message;
+ retry(retry.cfId, retry.sequenceNumber);
+ break;
+
+ case COMPLETE:
+ complete();
+ break;
+
+ case SESSION_FAILED:
+ sessionFailed();
+ break;
+ }
+ }
+
+ /**
+ * Call back for connection success.
+ *
+ * When connected, session moves to preparing phase and sends prepare message.
+ */
+ public void onConnect()
+ {
+ logger.debug("Connected. Sending prepare...");
+
+ // send prepare message
+ state(State.PREPARING);
+ PrepareMessage prepare = new PrepareMessage();
+ prepare.requests.addAll(requests);
+ for (StreamTransferTask task : transfers.values())
+ prepare.summaries.add(task.getSummary());
+ handler.sendMessage(prepare);
+
+ // if we don't need to prepare for receiving stream, start sending files immediately
+ if (requests.isEmpty())
+ {
+ logger.debug("Prepare complete. Start streaming files.");
+ startStreamingFiles();
+ }
+ }
+
+ /**
+ * Call back for handling exception during streaming.
+ *
+ * @param e thrown exception
+ */
+ public void onError(Throwable e)
+ {
+ state(State.FAILED);
+
+ logger.error("Streaming error occurred", e);
+ // send session failure message
+ handler.sendMessage(new SessionFailedMessage());
+ // fail session
+ streamResult.handleSessionComplete(this);
+ }
+
+ /**
+ * Prepare this session for sending/receiving files.
+ */
+ public void prepare(Collection<StreamRequest> requests, Collection<StreamSummary> summaries)
+ {
+ logger.debug("Start preparing this session (" + requests.size() + " requests, " + summaries.size() + " columnfamilies receiving)");
+ // prepare tasks
+ state(State.PREPARING);
+ for (StreamRequest request : requests)
+ addTransferRanges(request.keyspace, request.ranges, request.columnFamilies, true); // always flush on stream request
+ for (StreamSummary summary : summaries)
+ prepareReceiving(summary);
+
+ // send back prepare message if prepare message contains stream request
+ if (!requests.isEmpty())
+ {
+ PrepareMessage prepare = new PrepareMessage();
+ for (StreamTransferTask task : transfers.values())
+ prepare.summaries.add(task.getSummary());
+ handler.sendMessage(prepare);
+ }
+
+ // if there are files to stream
+ if (!maybeCompleted())
+ {
+ logger.debug("Prepare complete. Start streaming files.");
+ startStreamingFiles();
+ }
+ }
+
+ /**
+ * Call back after sending FileMessageHeader.
+ *
+ * @param header sent header
+ */
+ public void fileSent(FileMessageHeader header)
+ {
+ StreamingMetrics.totalOutgoingBytes.inc(header.size());
+ metrics.outgoingBytes.inc(header.size());
+ transfers.get(header.cfId).complete(header.sequenceNumber);
+ }
+
+ /**
+ * Call back after receiving FileMessageHeader.
+ *
+ * @param message received file
+ */
+ public void receive(FileMessage message)
+ {
+ StreamingMetrics.totalIncomingBytes.inc(message.header.size());
+ metrics.incomingBytes.inc(message.header.size());
+ receivers.get(message.header.cfId).receive(message.sstable);
+ }
+
+ public void progress(Descriptor desc, ProgressInfo.Direction direction, long bytes, long total)
+ {
+ ProgressInfo progress = new ProgressInfo(peer, desc.filenameFor(Component.DATA), direction, bytes, total);
+ streamResult.handleProgress(progress);
+ }
+
+ /**
+ * Call back on receiving {@code StreamMessage.Type.RETRY} message.
+ *
+ * @param cfId ColumnFamily ID
+ * @param sequenceNumber Sequence number to indicate which file to stream again
+ */
+ public void retry(UUID cfId, int sequenceNumber)
+ {
+ FileMessage message = transfers.get(cfId).createMessageForRetry(sequenceNumber);
+ handler.sendMessage(message);
+ }
+
+ /**
+ * Check if session is completed on receiving {@code StreamMessage.Type.COMPLETE} message.
+ */
+ public synchronized void complete()
+ {
+ if (state == State.WAIT_COMPLETE)
+ {
+ state(State.COMPLETE);
+ handler.close();
+ streamResult.handleSessionComplete(this);
+ }
+ else
+ {
+ state(State.WAIT_COMPLETE);
+ }
+ }
+
+ /**
+ * Call back on receiving {@code StreamMessage.Type.SESSION_FAILED} message.
+ */
+ public synchronized void sessionFailed()
+ {
+ handler.close();
+ streamResult.handleSessionComplete(this);
+ }
+
+ public void doRetry(FileMessageHeader header, Throwable e)
+ {
+ // retry
+ retries++;
+ if (retries > DatabaseDescriptor.getMaxStreamingRetries())
+ onError(new IOException("Too many retries for " + header, e));
+ else
+ handler.sendMessage(new RetryMessage(header.cfId, header.sequenceNumber));
+ }
+
+ /**
+ * @return Current snapshot of this session info.
+ */
+ public SessionInfo getSessionInfo()
+ {
+ List<StreamSummary> receivingSummaries = Lists.newArrayList();
+ for (StreamTask receiver : receivers.values())
+ receivingSummaries.add(receiver.getSummary());
+ List<StreamSummary> transferSummaries = Lists.newArrayList();
+ for (StreamTask transfer : transfers.values())
+ transferSummaries.add(transfer.getSummary());
+ return new SessionInfo(peer, receivingSummaries, transferSummaries, state);
+ }
+
+ public synchronized void taskCompleted(StreamReceiveTask completedTask)
+ {
+ receivers.remove(completedTask.cfId);
+ maybeCompleted();
+ }
+
+ public synchronized void taskCompleted(StreamTransferTask completedTask)
+ {
+ transfers.remove(completedTask.cfId);
+ maybeCompleted();
+ }
+
+ public void onJoin(InetAddress endpoint, EndpointState epState) {}
+ public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
+ public void onAlive(InetAddress endpoint, EndpointState state) {}
+ public void onDead(InetAddress endpoint, EndpointState state) {}
+
+ public void onRemove(InetAddress endpoint)
+ {
+ convict(endpoint, Double.MAX_VALUE);
+ }
+
+ public void onRestart(InetAddress endpoint, EndpointState epState)
+ {
+ convict(endpoint, Double.MAX_VALUE);
+ }
+
+ public void convict(InetAddress endpoint, double phi)
+ {
+ if (!endpoint.equals(peer))
+ return;
+
+ // We want a higher confidence in the failure detection than usual because failing a streaming wrongly has a high cost.
+ if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold())
+ return;
+
+ state(State.FAILED);
+ streamResult.handleSessionComplete(this);
+ }
+
+ private boolean maybeCompleted()
+ {
+ boolean completed = receivers.isEmpty() && transfers.isEmpty();
+ if (completed)
+ {
+ if (state == State.WAIT_COMPLETE)
+ {
+ state(State.COMPLETE);
+ handler.close();
+ streamResult.handleSessionComplete(this);
+ }
+ else
+ {
+ // notify peer that this session is completed
+ handler.sendMessage(new CompleteMessage());
+ state(State.WAIT_COMPLETE);
+ }
+ }
+ return completed;
+ }
+
+
+ /**
+ * Flushes matching column families from the given keyspace, or all columnFamilies
+ * if the cf list is empty.
+ */
+ private void flushSSTables(Iterable<ColumnFamilyStore> stores)
+ {
+ logger.info("Flushing memtables for {}...", stores);
+ List<Future<?>> flushes = new ArrayList<>();
+ for (ColumnFamilyStore cfs : stores)
+ flushes.add(cfs.forceFlush());
+ FBUtilities.waitOnFutures(flushes);
+ }
+
+ private void prepareReceiving(StreamSummary summary)
+ {
+ logger.debug("prepare receiving " + summary);
+ if (summary.files > 0)
+ receivers.put(summary.cfId, new StreamReceiveTask(this, summary.cfId, summary.files, summary.totalSize));
+ }
+
+ private void startStreamingFiles()
+ {
+ streamResult.handleSessionPrepared(this);
+
+ state(State.STREAMING);
+ for (StreamTransferTask task : transfers.values())
+ {
+ if (task.getFileMessages().size() > 0)
+ handler.sendMessages(task.getFileMessages());
+ else
+ taskCompleted(task); // there is no file to send
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamState.java b/src/java/org/apache/cassandra/streaming/StreamState.java
new file mode 100644
index 0000000..db50c2a
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamState.java
@@ -0,0 +1,53 @@
+/*
+ * 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.streaming;
+
+import java.io.Serializable;
+import java.util.Set;
+import java.util.UUID;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+/**
+ * Current snapshot of streaming progress.
+ */
+public class StreamState implements Serializable
+{
+ public final UUID planId;
+ public final String description;
+ public final Set<SessionInfo> sessions;
+
+ public StreamState(UUID planId, String description, Set<SessionInfo> sessions)
+ {
+ this.planId = planId;
+ this.description = description;
+ this.sessions = sessions;
+ }
+
+ public boolean hasFailedSession()
+ {
+ return Iterables.any(sessions, new Predicate<SessionInfo>()
+ {
+ public boolean apply(SessionInfo session)
+ {
+ return session.isFailed();
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSummary.java b/src/java/org/apache/cassandra/streaming/StreamSummary.java
new file mode 100644
index 0000000..a31e333
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamSummary.java
@@ -0,0 +1,107 @@
+/*
+ * 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.streaming;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.UUID;
+
+import com.google.common.base.Objects;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+/**
+ * Summary of streaming.
+ */
+public class StreamSummary implements Serializable
+{
+ public static final IVersionedSerializer<StreamSummary> serializer = new StreamSummarySerializer();
+
+ public final UUID cfId;
+
+ /**
+ * Number of files to transfer. Can be 0 if nothing to transfer for some streaming request.
+ */
+ public final int files;
+ public final long totalSize;
+
+ public StreamSummary(UUID cfId, int files, long totalSize)
+ {
+ this.cfId = cfId;
+ this.files = files;
+ this.totalSize = totalSize;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ StreamSummary summary = (StreamSummary) o;
+ return files == summary.files && totalSize == summary.totalSize && cfId.equals(summary.cfId);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hashCode(cfId, files, totalSize);
+ }
+
+ @Override
+ public String toString()
+ {
+ final StringBuilder sb = new StringBuilder("StreamSummary{");
+ sb.append("path=").append(cfId);
+ sb.append(", files=").append(files);
+ sb.append(", totalSize=").append(totalSize);
+ sb.append('}');
+ return sb.toString();
+ }
+
+ public static class StreamSummarySerializer implements IVersionedSerializer<StreamSummary>
+ {
+ // arbitrary version is fine for UUIDSerializer for now...
+ public void serialize(StreamSummary summary, DataOutput out, int version) throws IOException
+ {
+ UUIDSerializer.serializer.serialize(summary.cfId, out, MessagingService.current_version);
+ out.writeInt(summary.files);
+ out.writeLong(summary.totalSize);
+ }
+
+ public StreamSummary deserialize(DataInput in, int version) throws IOException
+ {
+ UUID cfId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+ int files = in.readInt();
+ long totalSize = in.readLong();
+ return new StreamSummary(cfId, files, totalSize);
+ }
+
+ public long serializedSize(StreamSummary summary, int version)
+ {
+ long size = UUIDSerializer.serializer.serializedSize(summary.cfId, MessagingService.current_version);
+ size += TypeSizes.NATIVE.sizeof(summary.files);
+ size += TypeSizes.NATIVE.sizeof(summary.totalSize);
+ return size;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamTask.java b/src/java/org/apache/cassandra/streaming/StreamTask.java
new file mode 100644
index 0000000..9e9e06f
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamTask.java
@@ -0,0 +1,55 @@
+/*
+ * 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.streaming;
+
+import java.util.UUID;
+
+/**
+ * StreamTask is an abstraction of the streaming task performed over specific ColumnFamily.
+ */
+public abstract class StreamTask
+{
+ /** StreamSession that this task belongs */
+ protected final StreamSession session;
+
+ protected final UUID cfId;
+
+ protected StreamTask(StreamSession session, UUID cfId)
+ {
+ this.session = session;
+ this.cfId = cfId;
+ }
+
+ /**
+ * @return total number of files this task receives/streams.
+ */
+ public abstract int getTotalNumberOfFiles();
+
+ /**
+ * @return total bytes expected to receive
+ */
+ public abstract long getTotalSize();
+
+ /**
+ * @return StreamSummary that describes this task
+ */
+ public StreamSummary getSummary()
+ {
+ return new StreamSummary(cfId, getTotalNumberOfFiles(), getTotalSize());
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
new file mode 100644
index 0000000..ba2df03
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.streaming.messages.FileMessage;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamTransferTask sends sections of SSTable files in certain ColumnFamily.
+ */
+public class StreamTransferTask extends StreamTask
+{
+ private final AtomicInteger sequenceNumber = new AtomicInteger(0);
+
+ private final Map<Integer, FileMessage> files = new HashMap<>();
+
+ private long totalSize;
+
+ public StreamTransferTask(StreamSession session, UUID cfId)
+ {
+ super(session, cfId);
+ }
+
+ public void addTransferFile(SSTableReader sstable, long estimatedKeys, List<Pair<Long, Long>> sections)
+ {
+ assert sstable != null && cfId.equals(sstable.metadata.cfId);
+ FileMessage message = new FileMessage(sstable, sequenceNumber.getAndIncrement(), estimatedKeys, sections);
+ files.put(message.header.sequenceNumber, message);
+ totalSize += message.header.size();
+ }
+
+ /**
+ * Complete sending file.
+ *
+ * @param sequenceNumber sequence number of completed file transfer
+ */
+ public void complete(int sequenceNumber)
+ {
+ files.remove(sequenceNumber);
+ // all file sent, notify session this task is complete.
+ if (files.isEmpty())
+ session.taskCompleted(this);
+ }
+
+ public int getTotalNumberOfFiles()
+ {
+ return files.size();
+ }
+
+ public long getTotalSize()
+ {
+ return totalSize;
+ }
+
+ public Collection<FileMessage> getFileMessages()
+ {
+ return files.values();
+ }
+
+ public FileMessage createMessageForRetry(int sequenceNumber)
+ {
+ return files.get(sequenceNumber);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamWriter.java b/src/java/org/apache/cassandra/streaming/StreamWriter.java
new file mode 100644
index 0000000..04301ba
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamWriter.java
@@ -0,0 +1,158 @@
+/*
+ * 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.streaming;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+
+import com.google.common.util.concurrent.RateLimiter;
+import com.ning.compress.lzf.LZFOutputStream;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.util.DataIntegrityMetadata;
+import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamWriter writes given section of the SSTable to given channel.
+ */
+public class StreamWriter
+{
+ private static final int DEFAULT_CHUNK_SIZE = 64 * 1024;
+
+ protected final SSTableReader sstable;
+ protected final Collection<Pair<Long, Long>> sections;
+ protected final RateLimiter limiter = StreamManager.getRateLimiter();
+ protected final StreamSession session;
+
+ private OutputStream compressedOutput;
+
+ // allocate buffer to use for transfers only once
+ private byte[] transferBuffer;
+
+ public StreamWriter(SSTableReader sstable, Collection<Pair<Long, Long>> sections, StreamSession session)
+ {
+ this.session = session;
+ this.sstable = sstable;
+ this.sections = sections;
+ }
+
+ /**
+ * Stream file of specified sections to given channel.
+ *
+ * StreamWriter uses LZF compression on wire to decrease size to transfer.
+ *
+ * @param channel where this writes data to
+ * @throws IOException on any I/O error
+ */
+ public void write(WritableByteChannel channel) throws IOException
+ {
+ long totalSize = totalSize();
+ RandomAccessReader file = sstable.openDataReader();
+ ChecksumValidator validator = null;
+ if (new File(sstable.descriptor.filenameFor(Component.CRC)).exists())
+ validator = DataIntegrityMetadata.checksumValidator(sstable.descriptor);
+
+ transferBuffer = validator == null ? new byte[DEFAULT_CHUNK_SIZE] : new byte[validator.chunkSize];
+
+ // setting up data compression stream
+ compressedOutput = new LZFOutputStream(Channels.newOutputStream(channel));
+ long progress = 0L;
+
+ try
+ {
+ // stream each of the required sections of the file
+ for (Pair<Long, Long> section : sections)
+ {
+ long start = validator == null ? section.left : validator.chunkStart(section.left);
+ int skipBytes = (int) (section.left - start);
+ // seek to the beginning of the section
+ file.seek(start);
+ if (validator != null)
+ validator.seek(start);
+
+ // length of the section to read
+ long length = section.right - start;
+ // tracks write progress
+ long bytesTransferred = 0;
+ while (bytesTransferred < length)
+ {
+ long lastWrite = write(file, validator, skipBytes, length, bytesTransferred);
+ bytesTransferred += lastWrite;
+ progress += lastWrite;
+ session.progress(sstable.descriptor, ProgressInfo.Direction.OUT, progress, totalSize);
+ skipBytes = 0;
+ }
+
+ // make sure that current section is send
+ compressedOutput.flush();
+ }
+ }
+ finally
+ {
+ // no matter what happens close file
+ FileUtils.closeQuietly(file);
+ }
+
+ // release reference only when completed successfully
+ sstable.releaseReference();
+ }
+
+ protected long totalSize()
+ {
+ long size = 0;
+ for (Pair<Long, Long> section : sections)
+ size += section.right - section.left;
+ return size;
+ }
+
+ /**
+ * Sequentially read bytes from the file and write them to the output stream
+ *
+ * @param reader The file reader to read from
+ * @param validator validator to verify data integrity
+ * @param start number of bytes to skip transfer, but include for validation.
+ * @param length The full length that should be transferred
+ * @param bytesTransferred Number of bytes remaining to transfer
+ *
+ * @return Number of bytes transferred
+ *
+ * @throws java.io.IOException on any I/O error
+ */
+ protected long write(RandomAccessReader reader, ChecksumValidator validator, int start, long length, long bytesTransferred) throws IOException
+ {
+ int toTransfer = (int) Math.min(transferBuffer.length, length - bytesTransferred);
+ int minReadable = (int) Math.min(transferBuffer.length, reader.length() - reader.getFilePointer());
+
+ reader.readFully(transferBuffer, 0, minReadable);
+ if (validator != null)
+ validator.validate(transferBuffer, 0, minReadable);
+
+ limiter.acquire(toTransfer);
+ compressedOutput.write(transferBuffer, start, (toTransfer - start));
+
+ return toTransfer;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
index 8c2a7bc..3730b0e 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
@@ -23,33 +23,24 @@ import java.io.IOException;
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.Collections;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
import org.apache.cassandra.db.TypeSizes;
import org.apache.cassandra.dht.AbstractBounds;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.CompactEndpointSerializationHelper;
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.*;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.UUIDGen;
import org.apache.cassandra.utils.UUIDSerializer;
-
/**
* Task that make two nodes exchange (stream) some ranges (for a given table/cf).
* This handle the case where the local node is neither of the two nodes that
@@ -72,9 +63,9 @@ public class StreamingRepairTask implements Runnable
private final String tableName;
private final String cfName;
private final Collection<Range<Token>> ranges;
- private final IStreamCallback callback;
+ private final StreamEventHandler callback;
- private StreamingRepairTask(UUID id, InetAddress owner, InetAddress src, InetAddress dst, String tableName, String cfName, Collection<Range<Token>> ranges, IStreamCallback callback)
+ private StreamingRepairTask(UUID id, InetAddress owner, InetAddress src, InetAddress dst, String tableName, String cfName, Collection<Range<Token>> ranges, StreamEventHandler callback)
{
this.id = id;
this.owner = owner;
@@ -121,21 +112,15 @@ public class StreamingRepairTask implements Runnable
private void initiateStreaming()
{
- ColumnFamilyStore cfstore = Table.open(tableName).getColumnFamilyStore(cfName);
- try
- {
- logger.info(String.format("[streaming task #%s] Performing streaming repair of %d ranges with %s", id, ranges.size(), dst));
- Collection<ColumnFamilyStore> cfses = Collections.singleton(cfstore);
- // send ranges to the remote node
- StreamOutSession outsession = StreamOutSession.create(tableName, dst, callback);
- StreamOut.transferRanges(outsession, cfses, ranges, OperationType.AES, false);
- // request ranges from the remote node
- StreamIn.requestRanges(dst, tableName, cfses, ranges, callback, OperationType.AES);
- }
- catch(Exception e)
- {
- throw new RuntimeException("Streaming repair failed", e);
- }
+ logger.info(String.format("[streaming task #%s] Performing streaming repair of %d ranges with %s", id, ranges.size(), dst));
+ StreamResultFuture op = new StreamPlan("Repair")
+ .flushBeforeTransfer(true)
+ // request ranges from the remote node
+ .requestRanges(dst, tableName, ranges, cfName)
+ // send ranges to the remote node
+ .transferRanges(dst, tableName, ranges, cfName)
+ .execute();
+ op.addEventListener(callback);
}
private void forwardToSource()
@@ -147,45 +132,34 @@ public class StreamingRepairTask implements Runnable
MessagingService.instance().sendOneWay(msg, src);
}
- private static IStreamCallback makeReplyingCallback(final InetAddress taskOwner, final UUID taskId)
+ private static StreamEventHandler makeReplyingCallback(final InetAddress taskOwner, final UUID taskId)
{
- return new IStreamCallback()
+ return new StreamEventHandler()
{
- // we expect one callback for the receive, and one for the send
- private final AtomicInteger outstanding = new AtomicInteger(2);
-
- public void onSuccess()
+ public void onSuccess(StreamState finalState)
{
- if (outstanding.decrementAndGet() > 0)
- return; // waiting on more calls
-
StreamingRepairResponse.reply(taskOwner, taskId);
}
- public void onFailure() {}
+ public void onFailure(Throwable t) {}
+ public void handleStreamEvent(StreamEvent event) {}
};
}
// wrap a given callback so as to unregister the streaming repair task on completion
- private static IStreamCallback wrapCallback(final Runnable callback, final UUID taskid, final boolean isLocalTask)
+ private static StreamEventHandler wrapCallback(final Runnable callback, final UUID taskid, final boolean isLocalTask)
{
- return new IStreamCallback()
+ return new StreamEventHandler()
{
- // we expect one callback for the receive, and one for the send
- private final AtomicInteger outstanding = new AtomicInteger(isLocalTask ? 2 : 1);
-
- public void onSuccess()
+ public void onSuccess(StreamState finalState)
{
- if (outstanding.decrementAndGet() > 0)
- // waiting on more calls
- return;
-
tasks.remove(taskid);
if (callback != null)
callback.run();
}
- public void onFailure() {}
+ public void onFailure(Throwable t) {}
+ public void handleStreamEvent(StreamEvent event) {}
};
}
@@ -220,7 +194,10 @@ public class StreamingRepairTask implements Runnable
logger.info(String.format("[streaming task #%s] task succeeded", task.id));
if (task.callback != null)
- task.callback.onSuccess();
+ {
+ // TODO null
+ task.callback.onSuccess(null);
+ }
}
private static void reply(InetAddress remote, UUID taskid)
[4/5] Streaming 2.0
Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/FileStreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/FileStreamTask.java b/src/java/org/apache/cassandra/streaming/FileStreamTask.java
deleted file mode 100644
index 04890ba..0000000
--- a/src/java/org/apache/cassandra/streaming/FileStreamTask.java
+++ /dev/null
@@ -1,291 +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.streaming;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.util.concurrent.Uninterruptibles;
-import com.ning.compress.lzf.LZFOutputStream;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.util.DataIntegrityMetadata;
-import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.metrics.StreamingMetrics;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.Throttle;
-import org.apache.cassandra.utils.WrappedRunnable;
-
-public class FileStreamTask extends WrappedRunnable
-{
- private static final Logger logger = LoggerFactory.getLogger(FileStreamTask.class);
-
- private static final int DEFAULT_CHUNK_SIZE = 64 * 1024;
- public static final int MAX_CONNECT_ATTEMPTS = 4;
-
- protected final StreamHeader header;
- protected final InetAddress to;
-
- // communication socket
- protected Socket socket;
- // socket's output/input stream
- private OutputStream output;
- private OutputStream compressedoutput;
- private DataInputStream input;
- // allocate buffer to use for transfers only once
- private byte[] transferBuffer;
- // outbound global throughput limiter
- protected final Throttle throttle;
- private final StreamReplyVerbHandler handler = new StreamReplyVerbHandler();
- protected final StreamingMetrics metrics;
-
- public FileStreamTask(StreamHeader header, InetAddress to)
- {
- this.header = header;
- this.to = to;
- this.throttle = new Throttle(toString(), new Throttle.ThroughputFunction()
- {
- /** @return Instantaneous throughput target in bytes per millisecond. */
- public int targetThroughput()
- {
- if (DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() < 1)
- // throttling disabled
- return 0;
- // total throughput
- int totalBytesPerMS = DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() * 1024 * 1024 / 8 / 1000;
- // per stream throughput (target bytes per MS)
- return totalBytesPerMS / Math.max(1, (int)StreamingMetrics.activeStreamsOutbound.count());
- }
- });
- metrics = StreamingMetrics.get(to);
- }
-
- public void runMayThrow() throws IOException
- {
- try
- {
- connectAttempt();
- // successfully connected: stream.
- // (at this point, if we fail, it is the receiver's job to re-request)
- stream();
-
- StreamOutSession session = StreamOutSession.get(header.sessionId);
- if (session == null)
- {
- logger.info("Found no stream out session at end of file stream task - this is expected if the receiver went down");
- }
- else if (session.getFiles().size() == 0)
- {
- // we are the last of our kind, receive the final confirmation before closing
- receiveReply();
- logger.info("Finished streaming session to {}", to);
- }
- }
- catch (IOException e)
- {
- StreamOutSession session = StreamOutSession.get(header.sessionId);
- if (session != null)
- session.close(false);
- throw e;
- }
- finally
- {
- try
- {
- close();
- }
- catch (IOException e)
- {
- if (logger.isDebugEnabled())
- logger.debug("error closing socket", e);
- }
- }
- if (logger.isDebugEnabled())
- logger.debug("Done streaming " + header.file);
- }
-
- /**
- * Stream file by it's sections specified by this.header
- * @throws IOException on any I/O error
- */
- protected void stream() throws IOException
- {
- ByteBuffer headerBuffer = MessagingService.instance().constructStreamHeader(header, false, MessagingService.instance().getVersion(to));
- // write header (this should not be compressed for compatibility with other messages)
- output.write(ByteBufferUtil.getArray(headerBuffer));
-
- if (header.file == null)
- return;
-
- // try to skip kernel page cache if possible
- RandomAccessReader file = RandomAccessReader.open(new File(header.file.getFilename()));
- Descriptor desc = Descriptor.fromFilename(header.file.getFilename());
- ChecksumValidator metadata = null;
- if (new File(desc.filenameFor(Component.CRC)).exists())
- metadata = DataIntegrityMetadata.checksumValidator(desc);
- transferBuffer = metadata == null ? new byte[DEFAULT_CHUNK_SIZE] : new byte[metadata.chunkSize];
-
- // setting up data compression stream
- compressedoutput = new LZFOutputStream(output);
-
- StreamingMetrics.activeStreamsOutbound.inc();
- try
- {
- long totalBytesTransferred = 0;
- // stream each of the required sections of the file
- for (Pair<Long, Long> section : header.file.sections)
- {
- long start = metadata == null ? section.left : metadata.chunkStart(section.left);
- int skipBytes = (int) (section.left - start);
- // seek to the beginning of the section
- file.seek(start);
- if (metadata != null)
- metadata.seek(start);
-
- // length of the section to read
- long length = section.right - start;
- // tracks write progress
- long bytesTransferred = 0;
-
- while (bytesTransferred < length)
- {
- long lastWrite = write(file, metadata, skipBytes, length, bytesTransferred);
- bytesTransferred += lastWrite;
- totalBytesTransferred += lastWrite;
- // store streaming progress
- header.file.progress += lastWrite;
- skipBytes = 0;
- }
-
- // make sure that current section is send
- compressedoutput.flush();
-
- if (logger.isDebugEnabled())
- logger.debug("Bytes transferred " + bytesTransferred + "/" + header.file.size);
- }
- StreamingMetrics.totalOutgoingBytes.inc(totalBytesTransferred);
- metrics.outgoingBytes.inc(totalBytesTransferred);
- // receive reply confirmation
- receiveReply();
- }
- finally
- {
- StreamingMetrics.activeStreamsOutbound.dec();
-
- // no matter what happens close file
- FileUtils.closeQuietly(file);
- }
- }
-
- public static void sendReply(MessageOut message, DataOutputStream out) throws IOException
- {
- out.writeInt(MessagingService.PROTOCOL_MAGIC);
- message.serialize(out, MessagingService.current_version);
- }
-
- protected void receiveReply() throws IOException
- {
- MessagingService.validateMagic(input.readInt());
- // since we reject streaming with different version, using current_version here is fine
- MessageIn message = MessageIn.read(input, MessagingService.current_version, -1);
- assert message.verb == MessagingService.Verb.STREAM_REPLY : "Non-reply message received on stream socket";
- handler.doVerb(message, -1);
- }
-
- /**
- * Sequentially read bytes from the file and write them to the output stream
- *
- * @param reader The file reader to read from
- * @param validator validator to verify data integrity
- * @param start number of bytes to skip transfer, but include for validation.
- * @param length The full length that should be transferred
- * @param bytesTransferred Number of bytes remaining to transfer
- *
- * @return Number of bytes transferred
- *
- * @throws IOException on any I/O error
- */
- protected long write(RandomAccessReader reader, ChecksumValidator validator, int start, long length, long bytesTransferred) throws IOException
- {
- int toTransfer = (int) Math.min(transferBuffer.length, length - bytesTransferred);
- int minReadable = (int) Math.min(transferBuffer.length, reader.length() - reader.getFilePointer());
-
- reader.readFully(transferBuffer, 0, minReadable);
- if (validator != null)
- validator.validate(transferBuffer, 0, minReadable);
-
- compressedoutput.write(transferBuffer, start, (toTransfer - start));
- throttle.throttleDelta(toTransfer);
-
- return toTransfer;
- }
-
- /**
- * Connects to the destination, with backoff for failed attempts.
- * TODO: all nodes on a cluster must currently use the same storage port
- * @throws IOException If all attempts fail.
- */
- private void connectAttempt() throws IOException
- {
- int attempts = 0;
- while (true)
- {
- try
- {
- socket = MessagingService.instance().getConnectionPool(to).newSocket();
- socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
- output = socket.getOutputStream();
- input = new DataInputStream(socket.getInputStream());
- break;
- }
- catch (IOException e)
- {
- if (++attempts >= MAX_CONNECT_ATTEMPTS)
- throw e;
-
- long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, attempts);
- logger.warn("Failed attempt " + attempts + " to connect to " + to + " to stream " + header.file + ". Retrying in " + waitms + " ms. (" + e + ")");
- Uninterruptibles.sleepUninterruptibly(waitms, TimeUnit.MILLISECONDS);
- }
- }
- }
-
- protected void close() throws IOException
- {
- if (output != null)
- output.close();
- }
-
- public String toString()
- {
- return String.format("FileStreamTask(session=%s, to=%s)", header.sessionId, to);
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/IStreamCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/IStreamCallback.java b/src/java/org/apache/cassandra/streaming/IStreamCallback.java
deleted file mode 100644
index f0d7754..0000000
--- a/src/java/org/apache/cassandra/streaming/IStreamCallback.java
+++ /dev/null
@@ -1,36 +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.streaming;
-
-/**
- * Callback interface for streaming session success/failure.
- */
-public interface IStreamCallback
-{
- /**
- * called when stream session is finished successfully.
- */
- public void onSuccess();
-
- /**
- * called when streaming somehow got in trouble.
- */
- public void onFailure();
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
deleted file mode 100644
index 92f5c7f..0000000
--- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
+++ /dev/null
@@ -1,200 +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.streaming;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.Collections;
-
-import com.google.common.base.Throwables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.ning.compress.lzf.LZFInputStream;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.db.compaction.CompactionController;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.sstable.SSTableWriter;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.metrics.StreamingMetrics;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.compress.CompressedInputStream;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.BytesReadTracker;
-import org.apache.cassandra.utils.Pair;
-
-public class IncomingStreamReader
-{
- private static final Logger logger = LoggerFactory.getLogger(IncomingStreamReader.class);
-
- protected final PendingFile localFile;
- protected final PendingFile remoteFile;
- protected final StreamInSession session;
- private final InputStream underliningStream;
- private final StreamingMetrics metrics;
-
- public IncomingStreamReader(StreamHeader header, Socket socket) throws IOException
- {
- socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
- InetAddress host = ((InetSocketAddress)socket.getRemoteSocketAddress()).getAddress();
- if (header.pendingFiles.isEmpty() && header.file != null)
- {
- // StreamInSession should be created already when receiving 2nd and after files
- if (!StreamInSession.hasSession(header.sessionId))
- {
- StreamReply reply = new StreamReply("", header.sessionId, StreamReply.Status.SESSION_FAILURE);
- FileStreamTask.sendReply(reply.createMessage(), new DataOutputStream(socket.getOutputStream()));
- throw new IOException("Session " + header.sessionId + " already closed.");
- }
- }
- session = StreamInSession.get(host, header.sessionId);
- session.setSocket(socket);
-
- session.addFiles(header.pendingFiles);
- // set the current file we are streaming so progress shows up in jmx
- session.setCurrentFile(header.file);
- session.setTable(header.table);
- // pendingFile gets the new context for the local node.
- remoteFile = header.file;
- localFile = remoteFile != null ? StreamIn.getContextMapping(remoteFile) : null;
-
- if (remoteFile != null)
- {
- if (remoteFile.compressionInfo == null)
- underliningStream = new LZFInputStream(socket.getInputStream());
- else
- underliningStream = new CompressedInputStream(socket.getInputStream(), remoteFile.compressionInfo);
- }
- else
- {
- underliningStream = null;
- }
- metrics = StreamingMetrics.get(host);
- }
-
- /**
- * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
- */
- public void read() throws IOException
- {
- if (remoteFile != null)
- {
- if (logger.isDebugEnabled())
- {
- logger.debug("Receiving stream");
- logger.debug("Creating file for {} with {} estimated keys",
- localFile.getFilename(),
- remoteFile.estimatedKeys);
- }
-
- assert remoteFile.estimatedKeys > 0;
- DataInput in = new DataInputStream(underliningStream);
- try
- {
- SSTableReader reader = streamIn(in, localFile, remoteFile);
- session.finished(remoteFile, reader);
- }
- catch (IOException ex)
- {
- retry();
- throw ex;
- }
- catch (RuntimeException e)
- {
- // if we encountered unexpected exception, fail this session
- session.close(false);
- throw e;
- }
- }
-
- session.closeIfFinished();
- }
-
- /**
- * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
- */
- private SSTableReader streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException
- {
- ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname);
- DecoratedKey key;
- SSTableWriter writer = new SSTableWriter(localFile.getFilename(), remoteFile.estimatedKeys);
- CompactionController controller = new CompactionController(cfs, Collections.<SSTableReader>emptySet(), Integer.MIN_VALUE);
-
- try
- {
- BytesReadTracker in = new BytesReadTracker(input);
- long totalBytesRead = 0;
-
- for (Pair<Long, Long> section : localFile.sections)
- {
- long length = section.right - section.left;
- // skip to beginning of section inside chunk
- if (remoteFile.compressionInfo != null)
- ((CompressedInputStream) underliningStream).position(section.left);
- long bytesRead = 0;
- while (bytesRead < length)
- {
- in.reset(0);
-
- key = StorageService.getPartitioner().decorateKey(ByteBufferUtil.readWithShortLength(in));
- writer.appendFromStream(key, cfs.metadata, in);
-
- cfs.invalidateCachedRow(key);
-
- bytesRead += in.getBytesRead();
- // when compressed, report total bytes of compressed chunks read since remoteFile.size is the sum of chunks transferred
- if (remoteFile.compressionInfo != null)
- remoteFile.progress = ((CompressedInputStream) underliningStream).getTotalCompressedBytesRead();
- else
- remoteFile.progress += in.getBytesRead();
- totalBytesRead += in.getBytesRead();
- }
- }
- StreamingMetrics.totalIncomingBytes.inc(totalBytesRead);
- metrics.incomingBytes.inc(totalBytesRead);
- return writer.closeAndOpenReader();
- }
- catch (Throwable e)
- {
- writer.abort();
- if (e instanceof IOException)
- throw (IOException) e;
- else
- throw Throwables.propagate(e);
- }
- finally
- {
- controller.close();
- }
- }
-
- private void retry()
- {
- /* Ask the source node to re-stream this file. */
- session.retry(remoteFile);
-
- /* Delete the orphaned file. */
- if (new File(localFile.getFilename()).isFile())
- FileUtils.deleteWithConfirm(new File(localFile.getFilename()));
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/OperationType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/OperationType.java b/src/java/org/apache/cassandra/streaming/OperationType.java
deleted file mode 100644
index da6e2f6..0000000
--- a/src/java/org/apache/cassandra/streaming/OperationType.java
+++ /dev/null
@@ -1,32 +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.streaming;
-
-/**
- * Streaming operation type.
- */
-public enum OperationType
-{
- AES,
- BOOTSTRAP,
- UNBOOTSTRAP,
- RESTORE_REPLICA_COUNT,
- BULK_LOAD,
- REBUILD
-}
-
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/PendingFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/PendingFile.java b/src/java/org/apache/cassandra/streaming/PendingFile.java
deleted file mode 100644
index 0d9ec35..0000000
--- a/src/java/org/apache/cassandra/streaming/PendingFile.java
+++ /dev/null
@@ -1,180 +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.streaming;
-
-import java.io.*;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.streaming.compress.CompressionInfo;
-import org.apache.cassandra.utils.Pair;
-
-/**
- * Represents portions of a file to be streamed between nodes.
- */
-public class PendingFile
-{
- public static final PendingFileSerializer serializer = new PendingFileSerializer();
-
- // NB: this reference is used to be able to release the acquired reference upon completion
- public final SSTableReader sstable;
-
- public final Descriptor desc;
- public final String component;
- public final List<Pair<Long, Long>> sections;
- public final OperationType type;
- /** total length of data to transfer */
- public final long size;
- /** estimated number of keys to transfer */
- public final long estimatedKeys;
- /** compression information. null if data is not compressed */
- public final CompressionInfo compressionInfo;
- public long progress;
-
- public PendingFile(Descriptor desc, PendingFile pf)
- {
- this(null, desc, pf.component, pf.sections, pf.type, pf.estimatedKeys, pf.compressionInfo);
- }
-
- public PendingFile(SSTableReader sstable, Descriptor desc, String component, List<Pair<Long,Long>> sections, OperationType type)
- {
- this(sstable, desc, component, sections, type, 0, null);
- }
-
- public PendingFile(SSTableReader sstable,
- Descriptor desc,
- String component,
- List<Pair<Long,Long>> sections,
- OperationType type,
- long estimatedKeys,
- CompressionInfo compressionInfo)
- {
- this.sstable = sstable;
- this.desc = desc;
- this.component = component;
- this.sections = sections;
- this.type = type;
-
- long tempSize = 0;
- if (compressionInfo == null)
- {
- for (Pair<Long, Long> section : sections)
- tempSize += section.right - section.left;
- }
- else
- {
- // calculate total length of transferring chunks
- for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
- tempSize += chunk.length + 4; // 4 bytes for CRC
- }
- size = tempSize;
-
- this.estimatedKeys = estimatedKeys;
- this.compressionInfo = compressionInfo;
- }
-
- public String getFilename()
- {
- return desc.filenameFor(component);
- }
-
- public boolean equals(Object o)
- {
- if (!(o instanceof PendingFile))
- return false;
-
- PendingFile rhs = (PendingFile)o;
- return getFilename().equals(rhs.getFilename());
- }
-
- public int hashCode()
- {
- return getFilename().hashCode();
- }
-
- public String toString()
- {
- return getFilename() + " sections=" + sections.size() + " progress=" + progress + "/" + size + " - " + progress*100/size + "%";
- }
-
- public static class PendingFileSerializer implements IVersionedSerializer<PendingFile>
- {
- public void serialize(PendingFile sc, DataOutput out, int version) throws IOException
- {
- if (sc == null)
- {
- out.writeUTF("");
- return;
- }
-
- out.writeUTF(sc.desc.filenameFor(sc.component));
- out.writeUTF(sc.component);
- out.writeInt(sc.sections.size());
- for (Pair<Long,Long> section : sc.sections)
- {
- out.writeLong(section.left);
- out.writeLong(section.right);
- }
- out.writeUTF(sc.type.name());
- out.writeLong(sc.estimatedKeys);
- CompressionInfo.serializer.serialize(sc.compressionInfo, out, version);
- }
-
- public PendingFile deserialize(DataInput in, int version) throws IOException
- {
- String filename = in.readUTF();
- if (filename.isEmpty())
- return null;
-
- Descriptor desc = Descriptor.fromFilename(filename);
- String component = in.readUTF();
- int count = in.readInt();
- List<Pair<Long,Long>> sections = new ArrayList<Pair<Long,Long>>(count);
- for (int i = 0; i < count; i++)
- sections.add(Pair.create(in.readLong(), in.readLong()));
- // this controls the way indexes are rebuilt when streaming in.
- OperationType type = OperationType.RESTORE_REPLICA_COUNT;
- type = OperationType.valueOf(in.readUTF());
- long estimatedKeys = in.readLong();
- CompressionInfo info = null;
- info = CompressionInfo.serializer.deserialize(in, version);
- return new PendingFile(null, desc, component, sections, type, estimatedKeys, info);
- }
-
- public long serializedSize(PendingFile pf, int version)
- {
- if (pf == null)
- return TypeSizes.NATIVE.sizeof("");
-
- long size = TypeSizes.NATIVE.sizeof(pf.desc.filenameFor(pf.component));
- size += TypeSizes.NATIVE.sizeof(pf.component);
- size += TypeSizes.NATIVE.sizeof(pf.sections.size());
- for (Pair<Long,Long> section : pf.sections)
- size += TypeSizes.NATIVE.sizeof(section.left) + TypeSizes.NATIVE.sizeof(section.right);
- size += TypeSizes.NATIVE.sizeof(pf.type.name());
- size += TypeSizes.NATIVE.sizeof(pf.estimatedKeys);
- size += CompressionInfo.serializer.serializedSize(pf.compressionInfo, version);
- return size;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/ProgressInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ProgressInfo.java b/src/java/org/apache/cassandra/streaming/ProgressInfo.java
new file mode 100644
index 0000000..d308ed0
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/ProgressInfo.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.Serializable;
+import java.net.InetAddress;
+
+import com.google.common.base.Objects;
+
+/**
+ * ProgressInfo contains file transfer progress.
+ */
+public class ProgressInfo implements Serializable
+{
+ /**
+ * Direction of the stream.
+ */
+ public static enum Direction
+ {
+ OUT(0),
+ IN(1);
+
+ public final byte code;
+
+ private Direction(int code)
+ {
+ this.code = (byte) code;
+ }
+
+ public static Direction fromByte(byte direction)
+ {
+ return direction == 0 ? OUT : IN;
+ }
+ }
+
+ public final InetAddress peer;
+ public final String fileName;
+ public final Direction direction;
+ public final long currentBytes;
+ public final long totalBytes;
+
+ public ProgressInfo(InetAddress peer, String fileName, Direction direction, long currentBytes, long totalBytes)
+ {
+ assert totalBytes > 0;
+
+ this.peer = peer;
+ this.fileName = fileName;
+ this.direction = direction;
+ this.currentBytes = currentBytes;
+ this.totalBytes = totalBytes;
+ }
+
+ /**
+ * @return true if file transfer is completed
+ */
+ public boolean isCompleted()
+ {
+ return currentBytes == totalBytes;
+ }
+
+ /**
+ * ProgressInfo is considered to be equal only when all attributes except currentBytes are equal.
+ */
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ ProgressInfo that = (ProgressInfo) o;
+
+ if (totalBytes != that.totalBytes) return false;
+ if (direction != that.direction) return false;
+ if (!fileName.equals(that.fileName)) return false;
+ return peer.equals(that.peer);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hashCode(peer, fileName, direction, totalBytes);
+ }
+
+ @Override
+ public String toString()
+ {
+ StringBuilder sb = new StringBuilder(fileName);
+ sb.append(" ").append(currentBytes);
+ sb.append("/").append(totalBytes).append(" bytes");
+ sb.append("(").append(currentBytes*100/totalBytes).append("%) ");
+ sb.append(direction == Direction.OUT ? "sent to " : "received from ");
+ sb.append(peer);
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/SessionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/SessionInfo.java b/src/java/org/apache/cassandra/streaming/SessionInfo.java
new file mode 100644
index 0000000..9532041
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/SessionInfo.java
@@ -0,0 +1,187 @@
+/*
+ * 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.streaming;
+
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+/**
+ * Stream session info.
+ */
+public final class SessionInfo implements Serializable
+{
+ public final InetAddress peer;
+ /** Immutable collection of receiving summaries */
+ public final Collection<StreamSummary> receivingSummaries;
+ /** Immutable collection of sending summaries*/
+ public final Collection<StreamSummary> sendingSummaries;
+ /** Current session state */
+ public final StreamSession.State state;
+
+ private final Map<String, ProgressInfo> receivingFiles;
+ private final Map<String, ProgressInfo> sendingFiles;
+
+ public SessionInfo(InetAddress peer,
+ Collection<StreamSummary> receivingSummaries,
+ Collection<StreamSummary> sendingSummaries,
+ StreamSession.State state)
+ {
+ this.peer = peer;
+ this.receivingSummaries = ImmutableSet.copyOf(receivingSummaries);
+ this.sendingSummaries = ImmutableSet.copyOf(sendingSummaries);
+ this.receivingFiles = new HashMap<>();
+ this.sendingFiles = new HashMap<>();
+ this.state = state;
+ }
+
+ public boolean isFailed()
+ {
+ return state == StreamSession.State.FAILED;
+ }
+
+ /**
+ * Update progress of receiving/sending file.
+ *
+ * @param newProgress new progress info
+ */
+ public synchronized void updateProgress(ProgressInfo newProgress)
+ {
+ assert peer.equals(newProgress.peer);
+
+ Map<String, ProgressInfo> currentFiles = newProgress.direction == ProgressInfo.Direction.IN
+ ? receivingFiles : sendingFiles;
+ currentFiles.put(newProgress.fileName, newProgress);
+ }
+
+ public Collection<ProgressInfo> getReceivingFiles()
+ {
+ return receivingFiles.values();
+ }
+
+ public Collection<ProgressInfo> getSendingFiles()
+ {
+ return sendingFiles.values();
+ }
+
+ /**
+ * @return total number of files already received.
+ */
+ public long getTotalFilesReceived()
+ {
+ return getTotalFilesCompleted(receivingFiles.values());
+ }
+
+ /**
+ * @return total number of files already sent.
+ */
+ public long getTotalFilesSent()
+ {
+ return getTotalFilesCompleted(sendingFiles.values());
+ }
+
+ /**
+ * @return total size(in bytes) already received.
+ */
+ public long getTotalSizeReceived()
+ {
+ return getTotalSizeInProgress(receivingFiles.values());
+ }
+
+ /**
+ * @return total size(in bytes) already sent.
+ */
+ public long getTotalSizeSent()
+ {
+ return getTotalSizeInProgress(sendingFiles.values());
+ }
+
+ /**
+ * @return total number of files to receive in the session
+ */
+ public long getTotalFilesToReceive()
+ {
+ return getTotalFiles(receivingSummaries);
+ }
+
+ /**
+ * @return total number of files to send in the session
+ */
+ public long getTotalFilesToSend()
+ {
+ return getTotalFiles(sendingSummaries);
+ }
+
+ /**
+ * @return total size(in bytes) to receive in the session
+ */
+ public long getTotalSizeToReceive()
+ {
+ return getTotalSizes(receivingSummaries);
+ }
+
+ /**
+ * @return total size(in bytes) to send in the session
+ */
+ public long getTotalSizeToSend()
+ {
+ return getTotalSizes(sendingSummaries);
+ }
+
+ private long getTotalSizeInProgress(Collection<ProgressInfo> files)
+ {
+ long total = 0;
+ for (ProgressInfo file : files)
+ total += file.currentBytes;
+ return total;
+ }
+
+ private long getTotalFiles(Collection<StreamSummary> summaries)
+ {
+ long total = 0;
+ for (StreamSummary summary : summaries)
+ total += summary.files;
+ return total;
+ }
+
+ private long getTotalSizes(Collection<StreamSummary> summaries)
+ {
+ long total = 0;
+ for (StreamSummary summary : summaries)
+ total += summary.totalSize;
+ return total;
+ }
+
+ private long getTotalFilesCompleted(Collection<ProgressInfo> files)
+ {
+ Iterable<ProgressInfo> completed = Iterables.filter(files, new Predicate<ProgressInfo>()
+ {
+ public boolean apply(ProgressInfo input)
+ {
+ return input.isCompleted();
+ }
+ });
+ return Iterables.size(completed);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamEvent.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamEvent.java b/src/java/org/apache/cassandra/streaming/StreamEvent.java
new file mode 100644
index 0000000..9af1fbd
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamEvent.java
@@ -0,0 +1,81 @@
+/*
+ * 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.streaming;
+
+import java.net.InetAddress;
+import java.util.UUID;
+
+public abstract class StreamEvent
+{
+ public static enum Type
+ {
+ STREAM_PREPARED,
+ STREAM_COMPLETE,
+ FILE_PROGRESS,
+ }
+
+ public final Type eventType;
+ public final UUID planId;
+
+ protected StreamEvent(Type eventType, UUID planId)
+ {
+ this.eventType = eventType;
+ this.planId = planId;
+ }
+
+ public static class SessionCompleteEvent extends StreamEvent
+ {
+ public final InetAddress peer;
+ public final boolean success;
+
+ public SessionCompleteEvent(StreamSession session)
+ {
+ super(Type.STREAM_COMPLETE, session.planId());
+ this.peer = session.peer;
+ this.success = session.isSuccess();
+ }
+ }
+
+ public static class ProgressEvent extends StreamEvent
+ {
+ public final ProgressInfo progress;
+
+ public ProgressEvent(UUID planId, ProgressInfo progress)
+ {
+ super(Type.FILE_PROGRESS, planId);
+ this.progress = progress;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "<ProgressEvent " + progress.toString() + ">";
+ }
+ }
+
+ public static class SessionPreparedEvent extends StreamEvent
+ {
+ public final SessionInfo session;
+
+ public SessionPreparedEvent(UUID planId, SessionInfo session)
+ {
+ super(Type.STREAM_PREPARED, planId);
+ this.session = session;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamEventHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamEventHandler.java b/src/java/org/apache/cassandra/streaming/StreamEventHandler.java
new file mode 100644
index 0000000..e2e84bb
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamEventHandler.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.streaming;
+
+import com.google.common.util.concurrent.FutureCallback;
+
+public interface StreamEventHandler extends FutureCallback<StreamState>
+{
+ /**
+ * Callback for various streaming events.
+ *
+ * @see StreamEvent.Type
+ * @param event Stream event.
+ */
+ void handleStreamEvent(StreamEvent event);
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamException.java b/src/java/org/apache/cassandra/streaming/StreamException.java
new file mode 100644
index 0000000..6e22db2
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.streaming;
+
+public class StreamException extends Throwable
+{
+ public final StreamState finalState;
+
+ public StreamException(StreamState finalState, String message)
+ {
+ super(message);
+ this.finalState = finalState;
+ }
+
+ public StreamException(StreamState finalState, String message, Throwable cause)
+ {
+ super(message, cause);
+ this.finalState = finalState;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamHeader.java b/src/java/org/apache/cassandra/streaming/StreamHeader.java
deleted file mode 100644
index a0bf832..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamHeader.java
+++ /dev/null
@@ -1,94 +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.streaming;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.*;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDSerializer;
-
-public class StreamHeader
-{
- public static final IVersionedSerializer<StreamHeader> serializer = new StreamHeaderSerializer();
-
- public final String table;
-
- /** file being sent on initial stream */
- public final PendingFile file;
-
- /** session ID */
- public final UUID sessionId;
-
- /** files to add to the session */
- public final Collection<PendingFile> pendingFiles;
-
- public StreamHeader(String table, UUID sessionId, PendingFile file)
- {
- this(table, sessionId, file, Collections.<PendingFile>emptyList());
- }
-
- public StreamHeader(String table, UUID sessionId, PendingFile first, Collection<PendingFile> pendingFiles)
- {
- this.table = table;
- this.sessionId = sessionId;
- this.file = first;
- this.pendingFiles = pendingFiles;
- }
-
- private static class StreamHeaderSerializer implements IVersionedSerializer<StreamHeader>
- {
- public void serialize(StreamHeader sh, DataOutput out, int version) throws IOException
- {
- out.writeUTF(sh.table);
- UUIDSerializer.serializer.serialize(sh.sessionId, out, MessagingService.current_version);
- PendingFile.serializer.serialize(sh.file, out, version);
- out.writeInt(sh.pendingFiles.size());
- for (PendingFile file : sh.pendingFiles)
- PendingFile.serializer.serialize(file, out, version);
- }
-
- public StreamHeader deserialize(DataInput in, int version) throws IOException
- {
- String table = in.readUTF();
- UUID sessionId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
- PendingFile file = PendingFile.serializer.deserialize(in, version);
- int size = in.readInt();
-
- List<PendingFile> pendingFiles = new ArrayList<PendingFile>(size);
- for (int i = 0; i < size; i++)
- pendingFiles.add(PendingFile.serializer.deserialize(in, version));
- return new StreamHeader(table, sessionId, file, pendingFiles);
- }
-
- public long serializedSize(StreamHeader sh, int version)
- {
- long size = TypeSizes.NATIVE.sizeof(sh.table);
- size += TypeSizes.NATIVE.sizeof(sh.sessionId);
- size += PendingFile.serializer.serializedSize(sh.file, version);
- size += TypeSizes.NATIVE.sizeof(sh.pendingFiles.size());
- for (PendingFile file : sh.pendingFiles)
- size += PendingFile.serializer.serializedSize(file, version);
- return size;
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamIn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamIn.java b/src/java/org/apache/cassandra/streaming/StreamIn.java
deleted file mode 100644
index 588728c..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamIn.java
+++ /dev/null
@@ -1,89 +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.streaming;
-
-import java.net.InetAddress;
-import java.util.Collection;
-
-import org.apache.commons.lang.StringUtils;
-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.Table;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * for streaming data from other nodes in to this one.
- * Sends a STREAM_REQUEST Message to the source node(s), after which StreamOut on that side takes over.
- * See StreamOut for details.
- */
-public class StreamIn
-{
- private static final Logger logger = LoggerFactory.getLogger(StreamIn.class);
-
- /** Request ranges for all column families in the given keyspace. */
- public static void requestRanges(InetAddress source, String tableName, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
- {
- requestRanges(source, tableName, Table.open(tableName).getColumnFamilyStores(), ranges, callback, type);
- }
-
- /**
- * Request ranges to be transferred from specific CFs
- */
- public static void requestRanges(InetAddress source, String tableName, Collection<ColumnFamilyStore> columnFamilies, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
- {
- assert ranges.size() > 0;
-
- if (logger.isDebugEnabled())
- logger.debug("Requesting from {} ranges {}", source, StringUtils.join(ranges, ", "));
- StreamInSession session = StreamInSession.create(source, callback);
- StreamRequest srm = new StreamRequest(FBUtilities.getBroadcastAddress(),
- ranges,
- tableName,
- columnFamilies,
- session.getSessionId(),
- type);
- MessagingService.instance().sendOneWay(srm.createMessage(), source);
- }
-
- /** Translates remote files to local files by creating a local sstable per remote sstable. */
- public static PendingFile getContextMapping(PendingFile remote)
- {
- /* Create a local sstable for each remote sstable */
- Descriptor remotedesc = remote.desc;
- if (!remotedesc.isStreamCompatible())
- throw new UnsupportedOperationException(String.format("SSTable %s is not compatible with current version %s",
- remote.getFilename(), Descriptor.Version.CURRENT));
-
- // new local sstable
- Table table = Table.open(remotedesc.ksname);
- ColumnFamilyStore cfStore = table.getColumnFamilyStore(remotedesc.cfname);
- Directories.DataDirectory localDir = cfStore.directories.getLocationCapableOfSize(remote.size);
- if (localDir == null)
- throw new RuntimeException("Insufficient disk space to store " + remote.size + " bytes");
- Descriptor localdesc = Descriptor.fromFilename(cfStore.getTempSSTablePath(cfStore.directories.getLocationForDisk(localDir)));
-
- return new PendingFile(localdesc, remote);
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamInSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamInSession.java b/src/java/org/apache/cassandra/streaming/StreamInSession.java
deleted file mode 100644
index 370183f..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamInSession.java
+++ /dev/null
@@ -1,251 +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.streaming;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.util.*;
-import java.util.concurrent.ConcurrentMap;
-
-import org.cliffc.high_scale_lib.NonBlockingHashMap;
-import org.cliffc.high_scale_lib.NonBlockingHashSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.gms.FailureDetector;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDGen;
-
-/** each context gets its own StreamInSession. So there may be >1 Session per host */
-public class StreamInSession extends AbstractStreamSession
-{
- private static final Logger logger = LoggerFactory.getLogger(StreamInSession.class);
-
- private static final ConcurrentMap<UUID, StreamInSession> sessions = new NonBlockingHashMap<UUID, StreamInSession>();
-
- private final Set<PendingFile> files = new NonBlockingHashSet<PendingFile>();
- private final List<SSTableReader> readers = new ArrayList<SSTableReader>();
- private PendingFile current;
- private Socket socket;
- private volatile int retries;
-
- private StreamInSession(InetAddress host, UUID sessionId, IStreamCallback callback)
- {
- super(null, host, sessionId, callback);
- }
-
- public static StreamInSession create(InetAddress host, IStreamCallback callback)
- {
- StreamInSession session = new StreamInSession(host, UUIDGen.getTimeUUID(), callback);
- sessions.put(session.getSessionId(), session);
- return session;
- }
-
- public static StreamInSession get(InetAddress host, UUID sessionId)
- {
- StreamInSession session = sessions.get(sessionId);
- if (session == null)
- {
- StreamInSession possibleNew = new StreamInSession(host, sessionId, null);
- if ((session = sessions.putIfAbsent(sessionId, possibleNew)) == null)
- session = possibleNew;
- }
- return session;
- }
-
- public static boolean hasSession(UUID sessionId)
- {
- return sessions.get(sessionId) != null;
- }
-
- public void setCurrentFile(PendingFile file)
- {
- this.current = file;
- }
-
- public void setTable(String table)
- {
- this.table = table;
- }
-
- public void setSocket(Socket socket)
- {
- this.socket = socket;
- }
-
- public void addFiles(Collection<PendingFile> files)
- {
- for (PendingFile file : files)
- {
- if(logger.isDebugEnabled())
- logger.debug("Adding file {} to Stream Request queue", file.getFilename());
- this.files.add(file);
- }
- }
-
- public void finished(PendingFile remoteFile, SSTableReader reader) throws IOException
- {
- if (logger.isDebugEnabled())
- logger.debug("Finished {} (from {}). Sending ack to {}", new Object[] {remoteFile, getHost(), this});
-
- assert reader != null;
- readers.add(reader);
- files.remove(remoteFile);
- if (remoteFile.equals(current))
- current = null;
- StreamReply reply = new StreamReply(remoteFile.getFilename(), getSessionId(), StreamReply.Status.FILE_FINISHED);
- // send a StreamStatus message telling the source node it can delete this file
- sendMessage(reply.createMessage());
- logger.debug("ack {} sent for {}", reply, remoteFile);
- }
-
- public void retry(PendingFile remoteFile)
- {
- retries++;
- if (retries > DatabaseDescriptor.getMaxStreamingRetries())
- {
- logger.error(String.format("Failed streaming session %s from %s while receiving %s", getSessionId(), getHost().toString(), current),
- new IllegalStateException("Too many retries for " + remoteFile));
- close(false);
- return;
- }
- StreamReply reply = new StreamReply(remoteFile.getFilename(), getSessionId(), StreamReply.Status.FILE_RETRY);
- logger.info("Streaming of file {} for {} failed: requesting a retry.", remoteFile, this);
- try
- {
- sendMessage(reply.createMessage());
- }
- catch (IOException e)
- {
- logger.error("Sending retry message failed, closing session.", e);
- close(false);
- }
- }
-
- public void sendMessage(MessageOut<StreamReply> message) throws IOException
- {
- DataOutputStream out = new DataOutputStream(socket.getOutputStream());
- FileStreamTask.sendReply(message,
- out);
- out.flush();
- }
-
- public void closeIfFinished() throws IOException
- {
- if (files.isEmpty())
- {
- HashMap <ColumnFamilyStore, List<SSTableReader>> cfstores = new HashMap<ColumnFamilyStore, List<SSTableReader>>();
- try
- {
- for (SSTableReader sstable : readers)
- {
- assert sstable.getTableName().equals(table);
-
- // Acquire the reference (for secondary index building) before submitting the index build,
- // so it can't get compacted out of existence in between
- if (!sstable.acquireReference())
- throw new AssertionError("We shouldn't fail acquiring a reference on a sstable that has just been transferred");
-
- ColumnFamilyStore cfs = Table.open(sstable.getTableName()).getColumnFamilyStore(sstable.getColumnFamilyName());
- if (!cfstores.containsKey(cfs))
- cfstores.put(cfs, new ArrayList<SSTableReader>());
- cfstores.get(cfs).add(sstable);
- }
-
- // add sstables and build secondary indexes
- for (Map.Entry<ColumnFamilyStore, List<SSTableReader>> entry : cfstores.entrySet())
- {
- if (entry.getKey() != null)
- {
- entry.getKey().addSSTables(entry.getValue());
- entry.getKey().indexManager.maybeBuildSecondaryIndexes(entry.getValue(), entry.getKey().indexManager.allIndexesNames());
- }
- }
- }
- finally
- {
- for (List<SSTableReader> referenced : cfstores.values())
- SSTableReader.releaseReferences(referenced);
- }
-
- // send reply to source that we're done
- StreamReply reply = new StreamReply("", getSessionId(), StreamReply.Status.SESSION_FINISHED);
- logger.info("Finished streaming session {} from {}", getSessionId(), getHost());
- try
- {
- if (socket != null)
- FileStreamTask.sendReply(reply.createMessage(),
- new DataOutputStream(socket.getOutputStream()));
- else
- logger.debug("No socket to reply to {} with!", getHost());
- }
- finally
- {
- if (socket != null)
- socket.close();
- }
-
- close(true);
- }
- }
-
- protected void closeInternal(boolean success)
- {
- sessions.remove(sessionId);
- if (!success && FailureDetector.instance.isAlive(getHost()))
- {
- StreamReply reply = new StreamReply("", getSessionId(), StreamReply.Status.SESSION_FAILURE);
- MessagingService.instance().sendOneWay(reply.createMessage(), getHost());
- }
- }
-
- /** query method to determine which hosts are streaming to this node. */
- public static Set<InetAddress> getSources()
- {
- HashSet<InetAddress> set = new HashSet<InetAddress>();
- for (StreamInSession session : sessions.values())
- {
- set.add(session.getHost());
- }
- return set;
- }
-
- /** query the status of incoming files. */
- public static Set<PendingFile> getIncomingFiles(InetAddress host)
- {
- Set<PendingFile> set = new HashSet<PendingFile>();
- for (Map.Entry<UUID, StreamInSession> entry : sessions.entrySet())
- {
- StreamInSession session = entry.getValue();
- if (session.getHost().equals(host))
- {
- if (session.current != null)
- set.add(session.current);
- set.addAll(session.files);
- }
- }
- return set;
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamManager.java b/src/java/org/apache/cassandra/streaming/StreamManager.java
new file mode 100644
index 0000000..d64cc65
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamManager.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.RateLimiter;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * StreamManager manages currently running {@link StreamResultFuture}s and provides status of all operation invoked.
+ *
+ * All stream operation should be created through this class to track streaming status and progress.
+ */
+public class StreamManager implements StreamManagerMBean
+{
+ public static final StreamManager instance = new StreamManager();
+
+ private static final RateLimiter limiter = RateLimiter.create(Double.MAX_VALUE);
+
+ /**
+ * Gets streaming rate limiter.
+ * When stream_throughput_outbound_megabits_per_sec is 0, this returns rate limiter
+ * with the rate of Double.MAX_VALUE bytes per second.
+ * Rate unit is bytes per sec.
+ *
+ * @return RateLimiter with rate limit set
+ */
+ public static RateLimiter getRateLimiter()
+ {
+ double currentThroughput = DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() * 1024 * 1024 / 8 / 1000;
+ // if throughput is set to 0, throttling is disabled
+ if (currentThroughput == 0)
+ currentThroughput = Double.MAX_VALUE;
+ if (limiter.getRate() != currentThroughput)
+ limiter.setRate(currentThroughput);
+ return limiter;
+ }
+
+ /** Currently running stream plans. Removed after completion/failure. */
+ private final Map<UUID, StreamResultFuture> currentStreams = new NonBlockingHashMap<>();
+
+ public Set<StreamState> getCurrentStreams()
+ {
+ return Sets.newHashSet(Iterables.transform(currentStreams.values(), new Function<StreamResultFuture, StreamState>()
+ {
+ public StreamState apply(StreamResultFuture input)
+ {
+ return input.getCurrentState();
+ }
+ }));
+ }
+
+ public void register(final StreamResultFuture result)
+ {
+ // Make sure we remove the stream on completion (whether successful or not)
+ result.addListener(new Runnable()
+ {
+ public void run()
+ {
+ currentStreams.remove(result.planId);
+ }
+ }, MoreExecutors.sameThreadExecutor());
+
+ currentStreams.put(result.planId, result);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamManagerMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamManagerMBean.java b/src/java/org/apache/cassandra/streaming/StreamManagerMBean.java
new file mode 100644
index 0000000..eb6f6ae
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamManagerMBean.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.streaming;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+public interface StreamManagerMBean
+{
+ public static final String OBJECT_NAME = "org.apache.cassandra.net:type=StreamManager";
+
+ /**
+ * Returns the current state of all ongoing streams.
+ */
+ Set<StreamState> getCurrentStreams();
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamOut.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamOut.java b/src/java/org/apache/cassandra/streaming/StreamOut.java
deleted file mode 100644
index 709741b..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamOut.java
+++ /dev/null
@@ -1,187 +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.streaming;
-
-import java.net.InetAddress;
-import java.util.*;
-import java.util.concurrent.Future;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.streaming.compress.CompressionInfo;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-
-/**
- * This class handles streaming data from one node to another.
- *
- * The source node [the Out side] is always in charge of the streaming session. Streams may
- * be initiated either directly by the source via the methods in this class,
- * or on demand from the target (via StreamRequest).
- *
- * Files to stream are grouped into sessions, which can have callbacks associated
- * with them so that (for instance) we can mark a new node a full member of the
- * cluster after all the data it needs has been streamed.
- *
- * The source begins a session by sending
- * a Message with the stream bit flag in the Header turned on. Part of that Message
- * will include a StreamHeader that includes the files that will be streamed as part
- * of that session, as well as the first file-to-be-streamed. (Combining session list
- * and first file like this is inconvenient, but not as inconvenient as the old
- * three-part send-file-list, wait-for-ack, start-first-file dance.)
- *
- * This is done over a separate TCP connection to avoid blocking ordinary intra-node
- * traffic during the stream. So there is no Handler for the main stream of data --
- * when a connection sets the Stream bit, IncomingTcpConnection knows what to expect
- * without any further Messages.
- *
- * After each file, the target node [the In side] will send a StreamReply indicating success
- * (FILE_FINISHED) or failure (FILE_RETRY).
- *
- * When all files have been successfully transferred and integrated the target will
- * send an additional SESSION_FINISHED reply and the session is complete.
- *
- * For Stream requests (for bootstrap), one subtlety is that we always have to
- * create at least one stream reply, even if the list of files is empty, otherwise the
- * target has no way to know that it can stop waiting for an answer.
- *
- */
-public class StreamOut
-{
- private static final Logger logger = LoggerFactory.getLogger(StreamOut.class);
-
- /**
- * Stream the given ranges to the target endpoint from each CF in the given keyspace.
- */
- public static void transferRanges(InetAddress target, Table table, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
- {
- transferRanges(target, table, table.getColumnFamilyStores(), ranges, callback, type);
- }
-
- /**
- * Stream the given ranges to the target endpoint for provided CFs in the given keyspace.
- */
- public static void transferRanges(InetAddress target, Table table, Iterable<ColumnFamilyStore> cfses, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
- {
- StreamOutSession session = StreamOutSession.create(table.getName(), target, callback);
- transferRanges(session, cfses, ranges, type);
- }
-
- /**
- * Flushes matching column families from the given keyspace, or all columnFamilies
- * if the cf list is empty.
- */
- private static void flushSSTables(Iterable<ColumnFamilyStore> stores)
- {
- logger.info("Flushing memtables for {}...", stores);
- List<Future<?>> flushes = new ArrayList<Future<?>>();
- for (ColumnFamilyStore cfstore : stores)
- flushes.add(cfstore.forceFlush());
- FBUtilities.waitOnFutures(flushes);
- }
-
- /**
- * Stream the given ranges to the target endpoint from each of the given CFs.
- */
- public static void transferRanges(StreamOutSession session, Iterable<ColumnFamilyStore> cfses, Collection<Range<Token>> ranges, OperationType type)
- {
- transferRanges(session, cfses, ranges, type, true);
- }
-
- /**
- * Stream the given ranges to the target endpoint from each of the given CFs.
- */
- public static void transferRanges(StreamOutSession session,
- Iterable<ColumnFamilyStore> cfses,
- Collection<Range<Token>> ranges,
- OperationType type,
- boolean flushTables)
- {
- assert ranges.size() > 0;
- logger.info("Beginning transfer to {}", session.getHost());
- logger.debug("Ranges are {}", StringUtils.join(ranges, ","));
-
- if (flushTables)
- flushSSTables(cfses);
-
- List<SSTableReader> sstables = Lists.newLinkedList();
- for (ColumnFamilyStore cfStore : cfses)
- {
- List<AbstractBounds<RowPosition>> rowBoundsList = Lists.newLinkedList();
- for (Range<Token> range : ranges)
- rowBoundsList.add(range.toRowBounds());
- ColumnFamilyStore.ViewFragment view = cfStore.markReferenced(rowBoundsList);
- sstables.addAll(view.sstables);
- }
-
- transferSSTables(session, sstables, ranges, type);
- }
-
- /**
- * Low-level transfer of matching portions of a group of sstables from a single table to the target endpoint.
- * You should probably call transferRanges instead. This moreover assumes that references have been acquired on the sstables.
- */
- public static void transferSSTables(StreamOutSession session, Iterable<SSTableReader> sstables, Collection<Range<Token>> ranges, OperationType type)
- {
- List<PendingFile> pending = createPendingFiles(sstables, ranges, type);
-
- // Even if the list of pending files is empty, we need to initiate the transfer otherwise
- // the remote end will hang in cases where this was a requested transfer.
- session.addFilesToStream(pending);
- session.begin();
- }
-
- // called prior to sending anything.
- private static List<PendingFile> createPendingFiles(Iterable<SSTableReader> sstables, Collection<Range<Token>> ranges, OperationType type)
- {
- List<PendingFile> pending = new ArrayList<PendingFile>();
- for (SSTableReader sstable : sstables)
- {
- Descriptor desc = sstable.descriptor;
- List<Pair<Long,Long>> sections = sstable.getPositionsForRanges(ranges);
- if (sections.isEmpty())
- {
- // A reference was acquired on the sstable and we won't stream it
- sstable.releaseReference();
- continue;
- }
- CompressionInfo compression = null;
- if (sstable.compression)
- {
- compression = new CompressionInfo(sstable.getCompressionMetadata().getChunksForSections(sections),
- sstable.getCompressionMetadata().parameters);
- }
- pending.add(new PendingFile(sstable, desc, SSTable.COMPONENT_DATA, sections, type, sstable.estimatedKeysForRanges(ranges), compression));
- }
- logger.info("Stream context metadata {}, {} sstables.", pending, Iterables.size(sstables));
- return pending;
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamOutSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamOutSession.java b/src/java/org/apache/cassandra/streaming/StreamOutSession.java
deleted file mode 100644
index edc07ca..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamOutSession.java
+++ /dev/null
@@ -1,164 +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.streaming;
-
-import java.net.InetAddress;
-import java.util.*;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDGen;
-
-import org.cliffc.high_scale_lib.NonBlockingHashMap;
-
-/**
- * This class manages the streaming of multiple files one after the other.
- */
-public class StreamOutSession extends AbstractStreamSession
-{
- private static final Logger logger = LoggerFactory.getLogger(StreamOutSession.class);
-
- // one host may have multiple stream sessions.
- private static final ConcurrentMap<UUID, StreamOutSession> streams = new NonBlockingHashMap<UUID, StreamOutSession>();
-
- public static StreamOutSession create(String table, InetAddress host, IStreamCallback callback)
- {
- return create(table, host, UUIDGen.getTimeUUID(), callback);
- }
-
- public static StreamOutSession create(String table, InetAddress host, UUID sessionId)
- {
- return create(table, host, sessionId, null);
- }
-
- public static StreamOutSession create(String table, InetAddress host, UUID sessionId, IStreamCallback callback)
- {
- StreamOutSession session = new StreamOutSession(table, host, sessionId, callback);
- streams.put(sessionId, session);
- return session;
- }
-
- public static StreamOutSession get(UUID sessionId)
- {
- return streams.get(sessionId);
- }
-
- private final Map<String, PendingFile> files = new NonBlockingHashMap<String, PendingFile>();
-
- private volatile String currentFile;
-
- private StreamOutSession(String table, InetAddress host, UUID sessionId, IStreamCallback callback)
- {
- super(table, host, sessionId, callback);
- }
-
- public void addFilesToStream(List<PendingFile> pendingFiles)
- {
- for (PendingFile pendingFile : pendingFiles)
- {
- if (logger.isDebugEnabled())
- logger.debug("Adding file {} to be streamed.", pendingFile.getFilename());
- files.put(pendingFile.getFilename(), pendingFile);
- }
- }
-
- public void retry()
- {
- streamFile(files.get(currentFile));
- }
-
- private void streamFile(PendingFile pf)
- {
- if (logger.isDebugEnabled())
- logger.debug("Streaming {} ...", pf);
- currentFile = pf.getFilename();
- MessagingService.instance().stream(new StreamHeader(table, getSessionId(), pf), getHost());
- }
-
- public void startNext()
- {
- assert files.containsKey(currentFile);
- files.get(currentFile).sstable.releaseReference();
- files.remove(currentFile);
- Iterator<PendingFile> iter = files.values().iterator();
- if (iter.hasNext())
- streamFile(iter.next());
- }
-
- protected void closeInternal(boolean success)
- {
- // Release reference on last file (or any uncompleted ones)
- for (PendingFile file : files.values())
- file.sstable.releaseReference();
- streams.remove(sessionId);
- }
-
- /** convenience method for use when testing */
- void await() throws InterruptedException
- {
- while (streams.containsKey(sessionId))
- Thread.sleep(10);
- }
-
- public Collection<PendingFile> getFiles()
- {
- return files.values();
- }
-
- public static Set<InetAddress> getDestinations()
- {
- Set<InetAddress> hosts = new HashSet<InetAddress>();
- for (StreamOutSession session : streams.values())
- {
- hosts.add(session.getHost());
- }
- return hosts;
- }
-
- public static List<PendingFile> getOutgoingFiles(InetAddress host)
- {
- List<PendingFile> list = new ArrayList<PendingFile>();
- for (Map.Entry<UUID, StreamOutSession> entry : streams.entrySet())
- {
- StreamOutSession session = entry.getValue();
- if (session.getHost().equals(host))
- list.addAll(session.getFiles());
- }
- return list;
- }
-
- public void validateCurrentFile(String file)
- {
- if (!file.equals(currentFile))
- throw new IllegalStateException(String.format("target reports current file is %s but is %s", file, currentFile));
- }
-
- public void begin()
- {
- PendingFile first = files.isEmpty() ? null : files.values().iterator().next();
- currentFile = first == null ? null : first.getFilename();
- StreamHeader header = new StreamHeader(table, getSessionId(), first, files.values());
- logger.info("Streaming to {}", getHost());
- logger.debug("Files are {}", StringUtils.join(files.values(), ","));
- MessagingService.instance().stream(header, getHost());
- }
-}
[5/5] git commit: Streaming 2.0
Posted by sl...@apache.org.
Streaming 2.0
patch by Yukim; reviewed by slebresne for CASSANDRA-5286
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/51511697
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/51511697
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/51511697
Branch: refs/heads/trunk
Commit: 51511697254615b570f4162bbcd2baee23a234e9
Parents: 40b6c5d
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue May 28 20:28:13 2013 -0500
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jun 20 19:05:24 2013 +0200
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/cassandra/dht/BootStrapper.java | 19 +-
.../org/apache/cassandra/dht/RangeStreamer.java | 69 +--
.../cassandra/hadoop/BulkRecordWriter.java | 15 +-
.../cassandra/io/sstable/SSTableLoader.java | 145 ++---
.../net/IncomingStreamingConnection.java | 76 +++
.../cassandra/net/IncomingTcpConnection.java | 54 +-
.../apache/cassandra/net/MessagingService.java | 102 +---
.../cassandra/service/StorageService.java | 273 +++------
.../streaming/AbstractStreamSession.java | 114 ----
.../cassandra/streaming/ConnectionHandler.java | 309 ++++++++++
.../cassandra/streaming/FileStreamTask.java | 291 ---------
.../cassandra/streaming/IStreamCallback.java | 36 --
.../streaming/IncomingStreamReader.java | 200 -------
.../cassandra/streaming/OperationType.java | 32 -
.../apache/cassandra/streaming/PendingFile.java | 180 ------
.../cassandra/streaming/ProgressInfo.java | 110 ++++
.../apache/cassandra/streaming/SessionInfo.java | 187 ++++++
.../apache/cassandra/streaming/StreamEvent.java | 81 +++
.../cassandra/streaming/StreamEventHandler.java | 31 +
.../cassandra/streaming/StreamException.java | 35 ++
.../cassandra/streaming/StreamHeader.java | 94 ---
.../apache/cassandra/streaming/StreamIn.java | 89 ---
.../cassandra/streaming/StreamInSession.java | 251 --------
.../cassandra/streaming/StreamManager.java | 94 +++
.../cassandra/streaming/StreamManagerMBean.java | 32 +
.../apache/cassandra/streaming/StreamOut.java | 187 ------
.../cassandra/streaming/StreamOutSession.java | 164 -----
.../apache/cassandra/streaming/StreamPlan.java | 169 ++++++
.../cassandra/streaming/StreamReader.java | 120 ++++
.../cassandra/streaming/StreamReceiveTask.java | 95 +++
.../apache/cassandra/streaming/StreamReply.java | 91 ---
.../streaming/StreamReplyVerbHandler.java | 63 --
.../cassandra/streaming/StreamRequest.java | 174 ++----
.../streaming/StreamRequestVerbHandler.java | 47 --
.../cassandra/streaming/StreamResultFuture.java | 172 ++++++
.../cassandra/streaming/StreamSession.java | 591 +++++++++++++++++++
.../apache/cassandra/streaming/StreamState.java | 53 ++
.../cassandra/streaming/StreamSummary.java | 107 ++++
.../apache/cassandra/streaming/StreamTask.java | 55 ++
.../cassandra/streaming/StreamTransferTask.java | 83 +++
.../cassandra/streaming/StreamWriter.java | 158 +++++
.../streaming/StreamingRepairTask.java | 75 +--
.../cassandra/streaming/StreamingService.java | 110 ----
.../streaming/StreamingServiceMBean.java | 41 --
.../compress/CompressedFileStreamTask.java | 160 -----
.../compress/CompressedInputStream.java | 7 +-
.../compress/CompressedStreamReader.java | 112 ++++
.../compress/CompressedStreamWriter.java | 129 ++++
.../streaming/messages/CompleteMessage.java | 42 ++
.../streaming/messages/FileMessage.java | 106 ++++
.../streaming/messages/FileMessageHeader.java | 151 +++++
.../streaming/messages/PrepareMessage.java | 91 +++
.../streaming/messages/RetryMessage.java | 57 ++
.../messages/SessionFailedMessage.java | 42 ++
.../streaming/messages/StreamInitMessage.java | 115 ++++
.../streaming/messages/StreamMessage.java | 109 ++++
.../org/apache/cassandra/tools/BulkLoader.java | 151 ++---
.../org/apache/cassandra/tools/NodeCmd.java | 61 +-
.../org/apache/cassandra/tools/NodeProbe.java | 29 +-
.../apache/cassandra/dht/BootStrapperTest.java | 3 +-
.../apache/cassandra/service/RemoveTest.java | 38 +-
.../cassandra/streaming/BootstrapTest.java | 54 --
.../cassandra/streaming/SerializationsTest.java | 220 -------
.../cassandra/streaming/SessionInfoTest.java | 73 +++
.../apache/cassandra/streaming/StreamUtil.java | 46 --
.../streaming/StreamingTransferTest.java | 198 ++++---
.../compress/CompressedInputStreamTest.java | 5 +-
68 files changed, 4068 insertions(+), 3406 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7e1bf77..2b21d05 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -60,6 +60,7 @@
* cqlsh: Add row count to SELECT output (CASSANDRA-5636)
* Include a timestamp with all read commands to determine column expiration
(CASSANDRA-5149)
+ * Streaming 2.0 (CASSANDRA-5286)
1.2.6
* Fix cross-DC mutation forwarding (CASSANDRA-5632)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index 28615cf..55d82e1 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -22,6 +22,7 @@ import java.io.DataOutput;
import java.io.IOException;
import java.net.InetAddress;
import java.util.*;
+import java.util.concurrent.ExecutionException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -36,7 +37,6 @@ import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.locator.AbstractReplicationStrategy;
import org.apache.cassandra.locator.TokenMetadata;
import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.OperationType;
public class BootStrapper
{
@@ -63,7 +63,7 @@ public class BootStrapper
if (logger.isDebugEnabled())
logger.debug("Beginning bootstrap process");
- RangeStreamer streamer = new RangeStreamer(tokenMetadata, address, OperationType.BOOTSTRAP);
+ RangeStreamer streamer = new RangeStreamer(tokenMetadata, address, "Bootstrap");
streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
for (String table : Schema.instance.getNonSystemTables())
@@ -72,8 +72,19 @@ public class BootStrapper
streamer.addRanges(table, strategy.getPendingAddressRanges(tokenMetadata, tokens, address));
}
- streamer.fetch();
- StorageService.instance.finishBootstrapping();
+ try
+ {
+ streamer.fetchAsync().get();
+ StorageService.instance.finishBootstrapping();
+ }
+ catch (InterruptedException e)
+ {
+ throw new RuntimeException("Interrupted while waiting on boostrap to complete. Bootstrap will have to be restarted.");
+ }
+ catch (ExecutionException e)
+ {
+ throw new RuntimeException("Error during boostrap: " + e.getCause().getMessage(), e.getCause());
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java
index a0e1a93..2196f4d 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -19,14 +19,10 @@ package org.apache.cassandra.dht;
import java.net.InetAddress;
import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
-import org.apache.cassandra.streaming.IStreamCallback;
-import org.apache.cassandra.utils.FBUtilities;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -38,8 +34,9 @@ import org.apache.cassandra.gms.IFailureDetector;
import org.apache.cassandra.locator.AbstractReplicationStrategy;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.streaming.OperationType;
-import org.apache.cassandra.streaming.StreamIn;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.utils.FBUtilities;
/**
* Assists in streaming ranges to a node.
@@ -50,12 +47,10 @@ public class RangeStreamer
private final TokenMetadata metadata;
private final InetAddress address;
- private final OperationType opType;
+ private final String description;
private final Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = HashMultimap.create();
private final Set<ISourceFilter> sourceFilters = new HashSet<ISourceFilter>();
- // protected for testing.
- protected CountDownLatch latch;
- private Set<Range<Token>> completed = Collections.newSetFromMap(new ConcurrentHashMap<Range<Token>, Boolean>());
+ private final StreamPlan streamPlan;
/**
* A filter applied to sources to stream from when constructing a fetch map.
@@ -104,11 +99,12 @@ public class RangeStreamer
}
}
- public RangeStreamer(TokenMetadata metadata, InetAddress address, OperationType opType)
+ public RangeStreamer(TokenMetadata metadata, InetAddress address, String description)
{
this.metadata = metadata;
this.address = address;
- this.opType = opType;
+ this.description = description;
+ this.streamPlan = new StreamPlan(description);
}
public void addSourceFilter(ISourceFilter filter)
@@ -123,7 +119,7 @@ public class RangeStreamer
if (logger.isDebugEnabled())
{
for (Map.Entry<Range<Token>, InetAddress> entry: rangesForTable.entries())
- logger.debug(String.format("%s: range %s exists on %s", opType, entry.getKey(), entry.getValue()));
+ logger.debug(String.format("%s: range %s exists on %s", description, entry.getKey(), entry.getValue()));
}
for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForTable, sourceFilters).asMap().entrySet())
@@ -131,7 +127,7 @@ public class RangeStreamer
if (logger.isDebugEnabled())
{
for (Range r : entry.getValue())
- logger.debug(String.format("%s: range %s from source %s for table %s", opType, r, entry.getKey(), table));
+ logger.debug(String.format("%s: range %s from source %s for table %s", description, r, entry.getKey(), table));
}
toFetch.put(table, entry);
}
@@ -219,50 +215,19 @@ public class RangeStreamer
return toFetch;
}
- public void fetch()
+ public StreamResultFuture fetchAsync()
{
- latch = new CountDownLatch(toFetch.entries().size());
-
for (Map.Entry<String, Map.Entry<InetAddress, Collection<Range<Token>>>> entry : toFetch.entries())
{
- final String table = entry.getKey();
- final InetAddress source = entry.getValue().getKey();
- final Collection<Range<Token>> ranges = entry.getValue().getValue();
+ String keyspace = entry.getKey();
+ InetAddress source = entry.getValue().getKey();
+ Collection<Range<Token>> ranges = entry.getValue().getValue();
/* Send messages to respective folks to stream data over to me */
- IStreamCallback callback = new IStreamCallback()
- {
- public void onSuccess()
- {
- completed.addAll(ranges);
- latch.countDown();
- if (logger.isDebugEnabled())
- logger.debug(String.format("Removed %s/%s as a %s source; remaining is %s",
- source, table, opType, latch.getCount()));
- }
-
- public void onFailure()
- {
- latch.countDown();
- logger.warn("Streaming from " + source + " failed");
- }
- };
if (logger.isDebugEnabled())
- logger.debug("" + opType + "ing from " + source + " ranges " + StringUtils.join(ranges, ", "));
- StreamIn.requestRanges(source, table, ranges, callback, opType);
+ logger.debug("" + description + "ing from " + source + " ranges " + StringUtils.join(ranges, ", "));
+ streamPlan.requestRanges(source, keyspace, ranges);
}
- try
- {
- latch.await();
- for (Map.Entry<String, Map.Entry<InetAddress, Collection<Range<Token>>>> entry : toFetch.entries())
- {
- if (!completed.containsAll(entry.getValue().getValue()))
- throw new RuntimeException(String.format("Unable to fetch range %s for keyspace %s from any hosts", entry.getValue().getValue(), entry.getKey()));
- }
- }
- catch (InterruptedException e)
- {
- throw new AssertionError(e);
- }
+ return streamPlan.execute();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
index de6d2e0..704d19f 100644
--- a/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
@@ -23,6 +23,8 @@ import java.net.InetAddress;
import java.nio.ByteBuffer;
import java.net.UnknownHostException;
import java.util.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@@ -38,6 +40,7 @@ import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.io.sstable.SSTableLoader;
import org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter;
+import org.apache.cassandra.streaming.StreamState;
import org.apache.cassandra.thrift.*;
import org.apache.cassandra.utils.OutputHandler;
import org.apache.hadoop.conf.Configuration;
@@ -221,7 +224,7 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
if (writer != null)
{
writer.close();
- SSTableLoader.LoaderFuture future = loader.stream();
+ Future<StreamState> future = loader.stream();
while (true)
{
try
@@ -229,7 +232,7 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
future.get(1000, TimeUnit.MILLISECONDS);
break;
}
- catch (TimeoutException te)
+ catch (ExecutionException | TimeoutException te)
{
progress.progress();
}
@@ -238,12 +241,12 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
throw new IOException(e);
}
}
- if (future.hadFailures())
+ if (loader.getFailedHosts().size() > 0)
{
- if (future.getFailedHosts().size() > maxFailures)
- throw new IOException("Too many hosts failed: " + future.getFailedHosts());
+ if (loader.getFailedHosts().size() > maxFailures)
+ throw new IOException("Too many hosts failed: " + loader.getFailedHosts());
else
- logger.warn("Some hosts failed: " + future.getFailedHosts());
+ logger.warn("Some hosts failed: " + loader.getFailedHosts());
}
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index 56bb1d4..4d54cd9 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -22,30 +22,29 @@ import java.io.FilenameFilter;
import java.io.IOException;
import java.net.InetAddress;
import java.util.*;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
import org.apache.cassandra.config.Config;
-import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.streaming.*;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
/**
* Cassandra SSTable bulk loader.
* Load an externally created sstable into a cluster.
*/
-public class SSTableLoader
+public class SSTableLoader implements StreamEventHandler
{
private final File directory;
private final String keyspace;
private final Client client;
private final OutputHandler outputHandler;
+ private final Set<InetAddress> failedHosts = new HashSet<>();
static
{
@@ -109,110 +108,61 @@ public class SSTableLoader
return sstables;
}
- public LoaderFuture stream()
+ public StreamResultFuture stream()
{
return stream(Collections.<InetAddress>emptySet());
}
- public LoaderFuture stream(Set<InetAddress> toIgnore)
+ public StreamResultFuture stream(Set<InetAddress> toIgnore)
{
client.init(keyspace);
+ StreamPlan plan = new StreamPlan("Bulk Load");
Collection<SSTableReader> sstables = openSSTables();
if (sstables.isEmpty())
{
- outputHandler.output("No sstables to stream");
- return new LoaderFuture(0);
+ // return empty result
+ return plan.execute();
}
-
Map<InetAddress, Collection<Range<Token>>> endpointToRanges = client.getEndpointToRangesMap();
- outputHandler.output(String.format("Streaming revelant part of %s to %s", names(sstables), endpointToRanges.keySet()));
+ outputHandler.output(String.format("Streaming relevant part of %sto %s", names(sstables), endpointToRanges.keySet()));
- // There will be one streaming session by endpoint
- LoaderFuture future = new LoaderFuture(endpointToRanges.size());
for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : endpointToRanges.entrySet())
{
InetAddress remote = entry.getKey();
if (toIgnore.contains(remote))
- {
- future.latch.countDown();
continue;
- }
Collection<Range<Token>> ranges = entry.getValue();
- StreamOutSession session = StreamOutSession.create(keyspace, remote, new CountDownCallback(future, remote));
// transferSSTables assumes references have been acquired
SSTableReader.acquireReferences(sstables);
- StreamOut.transferSSTables(session, sstables, ranges, OperationType.BULK_LOAD);
- future.setPendings(remote, session.getFiles());
+ plan.transferFiles(remote, ranges, sstables);
}
- return future;
+ StreamResultFuture bulkResult = plan.execute();
+ bulkResult.addEventListener(this);
+ return bulkResult;
}
- public static class LoaderFuture implements Future<Void>
- {
- final CountDownLatch latch;
- final Map<InetAddress, Collection<PendingFile>> pendingFiles;
- private List<InetAddress> failedHosts = new ArrayList<InetAddress>();
-
- private LoaderFuture(int request)
- {
- latch = new CountDownLatch(request);
- pendingFiles = new HashMap<InetAddress, Collection<PendingFile>>();
- }
-
- private void setPendings(InetAddress remote, Collection<PendingFile> files)
- {
- pendingFiles.put(remote, new ArrayList(files));
- }
+ public void onSuccess(StreamState finalState) {}
+ public void onFailure(Throwable t) {}
- private void setFailed(InetAddress addr)
- {
- failedHosts.add(addr);
- }
-
- public List<InetAddress> getFailedHosts()
- {
- return failedHosts;
- }
-
- public boolean cancel(boolean mayInterruptIfRunning)
- {
- throw new UnsupportedOperationException("Cancellation is not yet supported");
- }
-
- public Void get() throws InterruptedException
- {
- latch.await();
- return null;
- }
-
- public Void get(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
- {
- if (latch.await(timeout, unit))
- return null;
- else
- throw new TimeoutException();
- }
-
- public boolean isCancelled()
- {
- // For now, cancellation is not supported, maybe one day...
- return false;
- }
-
- public boolean isDone()
- {
- return latch.getCount() == 0;
- }
-
- public boolean hadFailures()
+ public void handleStreamEvent(StreamEvent event)
+ {
+ if (event.eventType == StreamEvent.Type.FILE_PROGRESS)
{
- return failedHosts.size() > 0;
+ ProgressInfo progress = ((StreamEvent.ProgressEvent) event).progress;
+ StringBuilder sb = new StringBuilder("\r");
+ sb.append(progress.fileName);
+ sb.append(": ");
+ sb.append(progress.currentBytes).append("/").append(progress.totalBytes);
+ System.out.print(sb.toString());
+ if (progress.currentBytes == progress.totalBytes)
+ System.out.println();
}
-
- public Map<InetAddress, Collection<PendingFile>> getPendingFiles()
+ else if (event.eventType == StreamEvent.Type.STREAM_COMPLETE)
{
- return pendingFiles;
+ StreamEvent.SessionCompleteEvent se = (StreamEvent.SessionCompleteEvent) event;
+ if (!se.success)
+ failedHosts.add(se.peer);
}
}
@@ -224,34 +174,9 @@ public class SSTableLoader
return builder.toString();
}
- private class CountDownCallback implements IStreamCallback
+ public Set<InetAddress> getFailedHosts()
{
- private final InetAddress endpoint;
- private final LoaderFuture future;
-
- CountDownCallback(LoaderFuture future, InetAddress endpoint)
- {
- this.future = future;
- this.endpoint = endpoint;
- }
-
- public void onSuccess()
- {
- future.latch.countDown();
- outputHandler.debug(String.format("Streaming session to %s completed (waiting on %d outstanding sessions)", endpoint, future.latch.getCount()));
-
- // There could be race with stop being called twice but it should be ok
- if (future.latch.getCount() == 0)
- client.stop();
- }
-
- public void onFailure()
- {
- outputHandler.output(String.format("Streaming session to %s failed", endpoint));
- future.setFailed(endpoint);
- future.latch.countDown();
- client.stop();
- }
+ return failedHosts;
}
public static abstract class Client
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
new file mode 100644
index 0000000..dd5b7b4
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.net;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.net.Socket;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.messages.StreamInitMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+/**
+ * Thread to consume stream init messages.
+ */
+public class IncomingStreamingConnection extends Thread
+{
+ private static final Logger logger = LoggerFactory.getLogger(IncomingStreamingConnection.class);
+
+ private final int version;
+ private final Socket socket;
+
+ public IncomingStreamingConnection(int version, Socket socket)
+ {
+ super("stream-init " + socket.getRemoteSocketAddress());
+ this.version = version;
+ this.socket = socket;
+ }
+
+ @Override
+ public void run()
+ {
+ try
+ {
+ // streaming connections are per-session and have a fixed version. we can't do anything with a wrong-version stream connection, so drop it.
+ if (version != StreamMessage.CURRENT_VERSION)
+ throw new IOException(String.format("Received stream using protocol version %d (my version %d). Terminating connection", version, MessagingService.current_version));
+
+ DataInput input = new DataInputStream(socket.getInputStream());
+ StreamInitMessage init = StreamInitMessage.serializer.deserialize(input, version);
+
+ StreamSession.startReceivingStreamAsync(init.planId, init.description, socket, version);
+ }
+ catch (IOException e)
+ {
+ logger.debug("IOException reading from socket; closing", e);
+ try
+ {
+ socket.close();
+ }
+ catch (IOException e2)
+ {
+ logger.debug("error closing socket", e2);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
index 1977086..4130006 100644
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
@@ -24,24 +24,25 @@ import java.net.SocketException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.xerial.snappy.SnappyInputStream;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.util.FastByteArrayInputStream;
-import org.apache.cassandra.streaming.IncomingStreamReader;
-import org.apache.cassandra.streaming.StreamHeader;
-import org.xerial.snappy.SnappyInputStream;
public class IncomingTcpConnection extends Thread
{
private static final Logger logger = LoggerFactory.getLogger(IncomingTcpConnection.class);
+ private final int version;
+ private final boolean compressed;
private final Socket socket;
public InetAddress from;
- public IncomingTcpConnection(Socket socket)
+ public IncomingTcpConnection(int version, boolean compressed, Socket socket)
{
assert socket != null;
+ this.version = version;
+ this.compressed = compressed;
this.socket = socket;
if (DatabaseDescriptor.getInternodeRecvBufferSize() != null)
{
@@ -66,20 +67,10 @@ public class IncomingTcpConnection extends Thread
{
try
{
- // determine the connection type to decide whether to buffer
- DataInputStream in = new DataInputStream(socket.getInputStream());
- MessagingService.validateMagic(in.readInt());
- int header = in.readInt();
- boolean isStream = MessagingService.getBits(header, 3, 1) == 1;
- int version = MessagingService.getBits(header, 15, 8);
- logger.debug("Connection version {} from {}", version, socket.getInetAddress());
-
- if (isStream)
- handleStream(in, version);
- else if (version < MessagingService.VERSION_12)
- handleLegacyVersion(version);
+ if (version < MessagingService.VERSION_12)
+ handleLegacyVersion();
else
- handleModernVersion(version, header);
+ handleModernVersion();
}
catch (EOFException e)
{
@@ -96,7 +87,7 @@ public class IncomingTcpConnection extends Thread
}
}
- private void handleModernVersion(int version, int header) throws IOException
+ private void handleModernVersion() throws IOException
{
DataOutputStream out = new DataOutputStream(socket.getOutputStream());
out.writeInt(MessagingService.current_version);
@@ -105,7 +96,6 @@ public class IncomingTcpConnection extends Thread
DataInputStream in = new DataInputStream(socket.getInputStream());
int maxVersion = in.readInt();
from = CompactEndpointSerializationHelper.deserialize(in);
- boolean compressed = MessagingService.getBits(header, 2, 1) == 1;
if (compressed)
{
@@ -136,28 +126,11 @@ public class IncomingTcpConnection extends Thread
}
}
- private void handleLegacyVersion(int version)
+ private void handleLegacyVersion()
{
throw new UnsupportedOperationException("Unable to read obsolete message version " + version + "; the earliest version supported is 1.2.0");
}
- private void handleStream(DataInputStream input, int version) throws IOException
- {
- if (version == MessagingService.current_version)
- {
- int size = input.readInt();
- byte[] headerBytes = new byte[size];
- input.readFully(headerBytes);
- stream(StreamHeader.serializer.deserialize(new DataInputStream(new FastByteArrayInputStream(headerBytes)), version), input);
- }
- else
- {
- // streaming connections are per-session and have a fixed version. we can't do anything with a wrong-version stream connection, so drop it.
- logger.error("Received stream using protocol version {} (my version {}). Terminating connection",
- version, MessagingService.current_version);
- }
- }
-
private InetAddress receiveMessage(DataInputStream input, int version) throws IOException
{
int id;
@@ -204,9 +177,4 @@ public class IncomingTcpConnection extends Thread
logger.debug("error closing socket", e);
}
}
-
- private void stream(StreamHeader streamHeader, DataInputStream input) throws IOException
- {
- new IncomingStreamReader(streamHeader, socket).read();
- }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 1569faf..8e94f8e 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -17,13 +17,9 @@
*/
package org.apache.cassandra.net;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOError;
-import java.io.IOException;
+import java.io.*;
import java.lang.management.ManagementFactory;
import java.net.*;
-import java.nio.ByteBuffer;
import java.nio.channels.AsynchronousCloseException;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.ServerSocketChannel;
@@ -53,7 +49,6 @@ import org.apache.cassandra.gms.GossipDigestAck;
import org.apache.cassandra.gms.GossipDigestAck2;
import org.apache.cassandra.gms.GossipDigestSyn;
import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.locator.ILatencySubscriber;
import org.apache.cassandra.metrics.ConnectionMetrics;
import org.apache.cassandra.metrics.DroppedMessageMetrics;
@@ -63,7 +58,6 @@ import org.apache.cassandra.service.*;
import org.apache.cassandra.service.paxos.Commit;
import org.apache.cassandra.service.paxos.PrepareResponse;
import org.apache.cassandra.streaming.*;
-import org.apache.cassandra.streaming.compress.CompressedFileStreamTask;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.*;
import org.cliffc.high_scale_lib.NonBlockingHashMap;
@@ -92,8 +86,8 @@ public final class MessagingService implements MessagingServiceMBean
REQUEST_RESPONSE, // client-initiated reads and writes
@Deprecated STREAM_INITIATE,
@Deprecated STREAM_INITIATE_DONE,
- STREAM_REPLY,
- STREAM_REQUEST,
+ @Deprecated STREAM_REPLY,
+ @Deprecated STREAM_REQUEST,
RANGE_SLICE,
@Deprecated BOOTSTRAP_TOKEN,
TREE_REQUEST,
@@ -192,8 +186,6 @@ public final class MessagingService implements MessagingServiceMBean
put(Verb.MUTATION, RowMutation.serializer);
put(Verb.READ_REPAIR, RowMutation.serializer);
put(Verb.READ, ReadCommand.serializer);
- put(Verb.STREAM_REPLY, StreamReply.serializer);
- put(Verb.STREAM_REQUEST, StreamRequest.serializer);
put(Verb.RANGE_SLICE, RangeSliceCommand.serializer);
put(Verb.BOOTSTRAP_TOKEN, BootStrapper.StringSerializer.instance);
put(Verb.TREE_REQUEST, ActiveRepairService.TreeRequest.serializer);
@@ -637,34 +629,6 @@ public final class MessagingService implements MessagingServiceMBean
return iar;
}
- /**
- * Stream a file from source to destination. This is highly optimized
- * to not hold any of the contents of the file in memory.
- *
- * @param header Header contains file to stream and other metadata.
- * @param to endpoint to which we need to stream the file.
- */
-
- public void stream(StreamHeader header, InetAddress to)
- {
- DebuggableThreadPoolExecutor executor = streamExecutors.get(to);
- if (executor == null)
- {
- // Using a core pool size of 0 is important. See documentation of streamExecutors.
- executor = DebuggableThreadPoolExecutor.createWithMaximumPoolSize("Streaming to " + to, 1, 1, TimeUnit.SECONDS);
- DebuggableThreadPoolExecutor old = streamExecutors.putIfAbsent(to, executor);
- if (old != null)
- {
- executor.shutdown();
- executor = old;
- }
- }
-
- executor.execute(header.file == null || header.file.compressionInfo == null
- ? new FileStreamTask(header, to)
- : new CompressedFileStreamTask(header, to));
- }
-
public void register(ILatencySubscriber subcriber)
{
subscribers.add(subcriber);
@@ -763,44 +727,6 @@ public final class MessagingService implements MessagingServiceMBean
return packed >>> (start + 1) - count & ~(-1 << count);
}
- public ByteBuffer constructStreamHeader(StreamHeader streamHeader, boolean compress, int version)
- {
- int header = 0;
- // set compression bit.
- if (compress)
- header |= 4;
- // set streaming bit
- header |= 8;
- // Setting up the version bit
- header |= (version << 8);
-
- /* Adding the StreamHeader which contains the session Id along
- * with the pendingfile info for the stream.
- * | Session Id | Pending File Size | Pending File | Bool more files |
- * | No. of Pending files | Pending Files ... |
- */
- byte[] bytes;
- try
- {
- DataOutputBuffer buffer = new DataOutputBuffer();
- StreamHeader.serializer.serialize(streamHeader, buffer, version);
- bytes = buffer.getData();
- }
- catch (IOException e)
- {
- throw new RuntimeException(e);
- }
- assert bytes.length > 0;
-
- ByteBuffer buffer = ByteBuffer.allocate(4 + 4 + 4 + bytes.length);
- buffer.putInt(PROTOCOL_MAGIC);
- buffer.putInt(header);
- buffer.putInt(bytes.length);
- buffer.put(bytes);
- buffer.flip();
- return buffer;
- }
-
/**
* @return the last version associated with address, or @param version if this is the first such version
*/
@@ -885,9 +811,29 @@ public final class MessagingService implements MessagingServiceMBean
{
Socket socket = server.accept();
if (authenticate(socket))
- new IncomingTcpConnection(socket).start();
+ {
+ // determine the connection type to decide whether to buffer
+ DataInputStream in = new DataInputStream(socket.getInputStream());
+ MessagingService.validateMagic(in.readInt());
+ int header = in.readInt();
+ boolean isStream = MessagingService.getBits(header, 3, 1) == 1;
+ int version = MessagingService.getBits(header, 15, 8);
+ logger.debug("Connection version {} from {}", version, socket.getInetAddress());
+
+ if (isStream)
+ {
+ new IncomingStreamingConnection(version, socket).start();
+ }
+ else
+ {
+ boolean compressed = MessagingService.getBits(header, 2, 1) == 1;
+ new IncomingTcpConnection(version, compressed, socket).start();
+ }
+ }
else
+ {
socket.close();
+ }
}
catch (AsynchronousCloseException e)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f6becd0..5e30207 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -37,6 +37,8 @@ import javax.management.ObjectName;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.*;
import com.google.common.util.concurrent.AtomicDouble;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.Uninterruptibles;
import org.apache.log4j.Level;
@@ -214,6 +216,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
{
jmxObjectName = new ObjectName("org.apache.cassandra.db:type=StorageService");
mbs.registerMBean(this, jmxObjectName);
+ mbs.registerMBean(StreamManager.instance, new ObjectName(StreamManager.OBJECT_NAME));
}
catch (Exception e)
{
@@ -232,8 +235,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_COMMIT, new CommitVerbHandler());
// see BootStrapper for a summary of how the bootstrap verbs interact
- MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAM_REQUEST, new StreamRequestVerbHandler());
- MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAM_REPLY, new StreamReplyVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REPLICATION_FINISHED, new ReplicationFinishedVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REQUEST_RESPONSE, new ResponseVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.INTERNAL_RESPONSE, new ResponseVerbHandler());
@@ -253,10 +254,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SNAPSHOT, new SnapshotVerbHandler());
MessagingService.instance().registerVerbHandlers(MessagingService.Verb.ECHO, new EchoVerbHandler());
-
- // spin up the streaming service so it is available for jmx tools.
- if (StreamingService.instance == null)
- throw new RuntimeException("Streaming service is unavailable.");
}
public void registerDaemon(CassandraDaemon daemon)
@@ -792,7 +789,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
{
logger.info("rebuild from dc: {}", sourceDc == null ? "(any dc)" : sourceDc);
- RangeStreamer streamer = new RangeStreamer(tokenMetadata, FBUtilities.getBroadcastAddress(), OperationType.REBUILD);
+ RangeStreamer streamer = new RangeStreamer(tokenMetadata, FBUtilities.getBroadcastAddress(), "Rebuild");
streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
if (sourceDc != null)
streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc));
@@ -800,7 +797,20 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
for (String table : Schema.instance.getNonSystemTables())
streamer.addRanges(table, getLocalRanges(table));
- streamer.fetch();
+ try
+ {
+ streamer.fetchAsync().get();
+ }
+ catch (InterruptedException e)
+ {
+ throw new RuntimeException("Interrupted while waiting on rebuild streaming");
+ }
+ catch (ExecutionException e)
+ {
+ // This is used exclusively through JMX, so log the full trace but only throw a simple RTE
+ logger.error("Error while rebuilding node", e.getCause());
+ throw new RuntimeException("Error while rebuilding node: " + e.getCause().getMessage());
+ }
}
public void setStreamThroughputMbPerSec(int value)
@@ -1775,7 +1785,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
*/
private void restoreReplicaCount(InetAddress endpoint, final InetAddress notifyEndpoint)
{
- final Multimap<InetAddress, String> fetchSources = HashMultimap.create();
Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> rangesToFetch = HashMultimap.create();
final InetAddress myAddress = FBUtilities.getBroadcastAddress();
@@ -1792,40 +1801,37 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
Multimap<InetAddress, Range<Token>> sourceRanges = getNewSourceRanges(table, myNewRanges);
for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : sourceRanges.asMap().entrySet())
{
- fetchSources.put(entry.getKey(), table);
rangesToFetch.put(table, entry);
}
}
+ StreamPlan stream = new StreamPlan("Restore replica count");
for (final String table : rangesToFetch.keySet())
{
for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : rangesToFetch.get(table))
{
final InetAddress source = entry.getKey();
Collection<Range<Token>> ranges = entry.getValue();
- final IStreamCallback callback = new IStreamCallback()
- {
- public void onSuccess()
- {
- synchronized (fetchSources)
- {
- fetchSources.remove(source, table);
- if (fetchSources.isEmpty())
- sendReplicationNotification(notifyEndpoint);
- }
- }
-
- public void onFailure()
- {
- logger.warn("Streaming from " + source + " failed");
- onSuccess(); // calling onSuccess to send notification
- }
- };
if (logger.isDebugEnabled())
logger.debug("Requesting from " + source + " ranges " + StringUtils.join(ranges, ", "));
- StreamIn.requestRanges(source, table, ranges, callback, OperationType.RESTORE_REPLICA_COUNT);
+ stream.requestRanges(source, table, ranges);
}
}
+ StreamResultFuture future = stream.execute();
+ Futures.addCallback(future, new FutureCallback<StreamState>()
+ {
+ public void onSuccess(StreamState finalState)
+ {
+ sendReplicationNotification(notifyEndpoint);
+ }
+
+ public void onFailure(Throwable t)
+ {
+ logger.warn("Streaming to restore replica count failed", t);
+ // We still want to send the notification
+ sendReplicationNotification(notifyEndpoint);
+ }
+ });
}
// needs to be modified to accept either a table or ARS.
@@ -2694,17 +2700,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
setMode(Mode.LEAVING, "streaming data to other nodes", true);
- CountDownLatch latch = streamRanges(rangesToStream);
- CountDownLatch hintsLatch = streamHints();
+ Future<StreamState> streamSuccess = streamRanges(rangesToStream);
+ Future<StreamState> hintsSuccess = streamHints();
// wait for the transfer runnables to signal the latch.
logger.debug("waiting for stream aks.");
try
{
- latch.await();
- hintsLatch.await();
+ streamSuccess.get();
+ hintsSuccess.get();
}
- catch (InterruptedException e)
+ catch (ExecutionException | InterruptedException e)
{
throw new RuntimeException(e);
}
@@ -2713,10 +2719,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
onFinish.run();
}
- private CountDownLatch streamHints()
+ private Future<StreamState> streamHints()
{
if (HintedHandOffManager.instance.listEndpointsPendingHints().size() == 0)
- return new CountDownLatch(0);
+ return Futures.immediateFuture(null);
// gather all live nodes in the cluster that aren't also leaving
List<InetAddress> candidates = new ArrayList<InetAddress>(StorageService.instance.getTokenMetadata().cloneAfterAllLeft().getAllEndpoints());
@@ -2731,7 +2737,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
if (candidates.isEmpty())
{
logger.warn("Unable to stream hints since no live endpoints seen");
- return new CountDownLatch(0);
+ return Futures.immediateFuture(null);
}
else
{
@@ -2743,14 +2749,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
Token token = StorageService.getPartitioner().getMinimumToken();
List<Range<Token>> ranges = Collections.singletonList(new Range<Token>(token, token));
- CountDownLatch latch = new CountDownLatch(1);
- StreamOut.transferRanges(hintsDestinationHost,
- Table.open(Table.SYSTEM_KS),
- Collections.singletonList(Table.open(Table.SYSTEM_KS).getColumnFamilyStore(SystemTable.HINTS_CF)),
- ranges,
- new CountingDownStreamCallback(latch, hintsDestinationHost),
- OperationType.UNBOOTSTRAP);
- return latch;
+ return new StreamPlan("Hints").transferRanges(hintsDestinationHost,
+ Table.SYSTEM_KS,
+ ranges,
+ SystemTable.HINTS_CF)
+ .execute();
}
}
@@ -2812,25 +2815,19 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
if (relocator.streamsNeeded())
{
setMode(Mode.MOVING, "fetching new ranges and streaming old ranges", true);
-
- relocator.logStreamsMap("[Move->STREAMING]");
- CountDownLatch streamLatch = relocator.streams();
-
- relocator.logRequestsMap("[Move->FETCHING]");
- CountDownLatch fetchLatch = relocator.requests();
-
try
{
- streamLatch.await();
- fetchLatch.await();
+ relocator.stream().get();
}
- catch (InterruptedException e)
+ catch (ExecutionException | InterruptedException e)
{
- throw new RuntimeException("Interrupted latch while waiting for stream/fetch ranges to finish: " + e.getMessage());
+ throw new RuntimeException("Interrupted while waiting for stream/fetch ranges to finish: " + e.getMessage());
}
}
else
+ {
setMode(Mode.MOVING, "No ranges to fetch/stream", true);
+ }
setTokens(Collections.singleton(newToken)); // setting new token as we have everything settled
@@ -2840,8 +2837,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
private class RangeRelocator
{
- private Map<String, Multimap<InetAddress, Range<Token>>> rangesToFetch = new HashMap<String, Multimap<InetAddress, Range<Token>>>();
- private Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable = new HashMap<String, Multimap<Range<Token>, InetAddress>>();
+ private StreamPlan streamPlan = new StreamPlan("Bootstrap");
private RangeRelocator(Collection<Token> tokens, List<String> tables)
{
@@ -2856,15 +2852,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
// clone to avoid concurrent modification in calculateNaturalEndpoints
TokenMetadata tokenMetaClone = tokenMetadata.cloneOnlyTokenMap();
- for (String table : tables)
+ for (String keyspace : tables)
{
for (Token newToken : newTokens)
{
// replication strategy of the current keyspace (aka table)
- AbstractReplicationStrategy strategy = Table.open(table).getReplicationStrategy();
+ AbstractReplicationStrategy strategy = Table.open(keyspace).getReplicationStrategy();
// getting collection of the currently used ranges by this keyspace
- Collection<Range<Token>> currentRanges = getRangesForEndpoint(table, localAddress);
+ Collection<Range<Token>> currentRanges = getRangesForEndpoint(keyspace, localAddress);
// collection of ranges which this node will serve after move to the new token
Collection<Range<Token>> updatedRanges = strategy.getPendingAddressRanges(tokenMetadata, newToken, localAddress);
@@ -2895,51 +2891,39 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
// calculating endpoints to stream current ranges to if needed
// in some situations node will handle current ranges as part of the new ranges
- Multimap<Range<Token>, InetAddress> rangeWithEndpoints = HashMultimap.create();
-
+ Multimap<InetAddress, Range<Token>> endpointRanges = HashMultimap.create();
for (Range<Token> toStream : rangesPerTable.left)
{
Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaClone));
Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaCloneAllSettled));
logger.debug("Range:" + toStream + "Current endpoints: " + currentEndpoints + " New endpoints: " + newEndpoints);
- rangeWithEndpoints.putAll(toStream, Sets.difference(newEndpoints, currentEndpoints));
+ for (InetAddress address : Sets.difference(newEndpoints, currentEndpoints))
+ endpointRanges.put(address, toStream);
}
- // associating table with range-to-endpoints map
- rangesToStreamByTable.put(table, rangeWithEndpoints);
+ // stream ranges
+ for (InetAddress address : endpointRanges.keySet())
+ streamPlan.transferRanges(address, keyspace, endpointRanges.get(address));
+ // stream requests
Multimap<InetAddress, Range<Token>> workMap = RangeStreamer.getWorkMap(rangesToFetchWithPreferredEndpoints);
- rangesToFetch.put(table, workMap);
+ for (InetAddress address : workMap.keySet())
+ streamPlan.requestRanges(address, keyspace, workMap.get(address));
if (logger.isDebugEnabled())
- logger.debug("Table {}: work map {}.", table, workMap);
+ logger.debug("Table {}: work map {}.", keyspace, workMap);
}
}
}
- private void logStreamsMap(String prefix)
- {
- logger.debug("{} Work map: {}", prefix, rangesToStreamByTable);
- }
-
- private void logRequestsMap(String prefix)
+ public Future<StreamState> stream()
{
- logger.debug("{} Work map: {}", prefix, rangesToFetch);
+ return streamPlan.execute();
}
- private boolean streamsNeeded()
+ public boolean streamsNeeded()
{
- return !rangesToStreamByTable.isEmpty() || !rangesToFetch.isEmpty();
- }
-
- private CountDownLatch streams()
- {
- return streamRanges(rangesToStreamByTable);
- }
-
- private CountDownLatch requests()
- {
- return requestRanges(rangesToFetch);
+ return !streamPlan.isEmpty();
}
}
@@ -2995,25 +2979,19 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
if (relocator.streamsNeeded())
{
setMode(Mode.RELOCATING, "fetching new ranges and streaming old ranges", true);
-
- relocator.logStreamsMap("[Relocate->STREAMING]");
- CountDownLatch streamLatch = relocator.streams();
-
- relocator.logRequestsMap("[Relocate->FETCHING]");
- CountDownLatch fetchLatch = relocator.requests();
-
try
{
- streamLatch.await();
- fetchLatch.await();
+ relocator.stream().get();
}
- catch (InterruptedException e)
+ catch (ExecutionException | InterruptedException e)
{
throw new RuntimeException("Interrupted latch while waiting for stream/fetch ranges to finish: " + e.getMessage());
}
}
else
+ {
setMode(Mode.RELOCATING, "no new ranges to stream/fetch", true);
+ }
Collection<Token> currentTokens = SystemTable.updateLocalTokens(tokens, Collections.<Token>emptyList());
tokenMetadata.updateNormalTokens(currentTokens, FBUtilities.getBroadcastAddress());
@@ -3420,26 +3398,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
* Seed data to the endpoints that will be responsible for it at the future
*
* @param rangesToStreamByTable tables and data ranges with endpoints included for each
- * @return latch to count down
+ * @return async Future for whether stream was success
*/
- private CountDownLatch streamRanges(final Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable)
+ private Future<StreamState> streamRanges(final Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable)
{
// First, we build a list of ranges to stream to each host, per table
final Map<String, Map<InetAddress, List<Range<Token>>>> sessionsToStreamByTable = new HashMap<String, Map<InetAddress, List<Range<Token>>>>();
- // The number of stream out sessions we need to start, to be built up as we build sessionsToStreamByTable
- int sessionCount = 0;
-
for (Map.Entry<String, Multimap<Range<Token>, InetAddress>> entry : rangesToStreamByTable.entrySet())
{
+ String keyspace = entry.getKey();
Multimap<Range<Token>, InetAddress> rangesWithEndpoints = entry.getValue();
if (rangesWithEndpoints.isEmpty())
continue;
- final String table = entry.getKey();
-
Map<InetAddress, List<Range<Token>>> rangesPerEndpoint = new HashMap<InetAddress, List<Range<Token>>>();
-
for (final Map.Entry<Range<Token>, InetAddress> endPointEntry : rangesWithEndpoints.entries())
{
final Range<Token> range = endPointEntry.getKey();
@@ -3454,12 +3427,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
curRanges.add(range);
}
- sessionCount += rangesPerEndpoint.size();
- sessionsToStreamByTable.put(table, rangesPerEndpoint);
+ sessionsToStreamByTable.put(keyspace, rangesPerEndpoint);
}
- final CountDownLatch latch = new CountDownLatch(sessionCount);
-
+ StreamPlan streamPlan = new StreamPlan("Unbootstrap");
for (Map.Entry<String, Map<InetAddress, List<Range<Token>>>> entry : sessionsToStreamByTable.entrySet())
{
final String table = entry.getKey();
@@ -3471,90 +3442,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
final InetAddress newEndpoint = rangesEntry.getKey();
// TODO each call to transferRanges re-flushes, this is potentially a lot of waste
- StreamOut.transferRanges(newEndpoint,
- Table.open(table),
- ranges,
- new CountingDownStreamCallback(latch, newEndpoint),
- OperationType.UNBOOTSTRAP);
- }
- }
- return latch;
- }
-
- static class CountingDownStreamCallback implements IStreamCallback
- {
- private final CountDownLatch latch;
- private final InetAddress targetAddr;
-
- CountingDownStreamCallback(CountDownLatch latch, InetAddress targetAddr)
- {
- this.latch = latch;
- this.targetAddr = targetAddr;
- }
-
- public void onSuccess()
- {
- latch.countDown();
- }
-
- public void onFailure()
- {
- logger.warn("Streaming to " + targetAddr + " failed");
- onSuccess(); // calling onSuccess for latch countdown
- }
- }
-
- /**
- * Used to request ranges from endpoints in the ring (will block until all data is fetched and ready)
- * @param ranges ranges to fetch as map of the preferred address and range collection
- * @return latch to count down
- */
- private CountDownLatch requestRanges(final Map<String, Multimap<InetAddress, Range<Token>>> ranges)
- {
- final CountDownLatch latch = new CountDownLatch(ranges.keySet().size());
- for (Map.Entry<String, Multimap<InetAddress, Range<Token>>> entry : ranges.entrySet())
- {
- Multimap<InetAddress, Range<Token>> endpointWithRanges = entry.getValue();
-
- if (endpointWithRanges.isEmpty())
- {
- latch.countDown();
- continue;
- }
-
- final String table = entry.getKey();
- final Set<InetAddress> pending = new HashSet<InetAddress>(endpointWithRanges.keySet());
-
- // Send messages to respective folks to stream data over to me
- for (final InetAddress source: endpointWithRanges.keySet())
- {
- Collection<Range<Token>> toFetch = endpointWithRanges.get(source);
-
- final IStreamCallback callback = new IStreamCallback()
- {
- public void onSuccess()
- {
- pending.remove(source);
-
- if (pending.isEmpty())
- latch.countDown();
- }
-
- public void onFailure()
- {
- logger.warn("Streaming from " + source + " failed");
- onSuccess(); // calling onSuccess for latch countdown
- }
- };
-
- if (logger.isDebugEnabled())
- logger.debug("Requesting from " + source + " ranges " + StringUtils.join(toFetch, ", "));
-
- // sending actual request
- StreamIn.requestRanges(source, table, toFetch, callback, OperationType.BOOTSTRAP);
+ streamPlan.transferRanges(newEndpoint, table, ranges);
}
}
- return latch;
+ return streamPlan.execute();
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java b/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java
deleted file mode 100644
index 89fbf5f..0000000
--- a/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java
+++ /dev/null
@@ -1,114 +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.streaming;
-
-import java.net.InetAddress;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.LoggerFactory;
-import org.slf4j.Logger;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.gms.*;
-
-public abstract class AbstractStreamSession implements IEndpointStateChangeSubscriber, IFailureDetectionEventListener
-{
- private static final Logger logger = LoggerFactory.getLogger(AbstractStreamSession.class);
-
- protected final InetAddress host;
- protected final UUID sessionId;
- protected String table;
- protected final IStreamCallback callback;
- private final AtomicBoolean isClosed = new AtomicBoolean(false);
-
- protected AbstractStreamSession(String table, InetAddress host, UUID sessionId, IStreamCallback callback)
- {
- this.host = host;
- this.sessionId = sessionId;
- this.table = table;
- this.callback = callback;
- Gossiper.instance.register(this);
- FailureDetector.instance.registerFailureDetectionEventListener(this);
- }
-
- public UUID getSessionId()
- {
- return sessionId;
- }
-
- public InetAddress getHost()
- {
- return host;
- }
-
- public void close(boolean success)
- {
- if (!isClosed.compareAndSet(false, true))
- {
- logger.debug("Stream session {} already closed", getSessionId());
- return;
- }
-
- closeInternal(success);
-
- Gossiper.instance.unregister(this);
- FailureDetector.instance.unregisterFailureDetectionEventListener(this);
-
- logger.debug("closing with status " + success);
- if (callback != null)
- {
- if (success)
- callback.onSuccess();
- else
- callback.onFailure();
- }
- }
-
- protected abstract void closeInternal(boolean success);
-
- public void onJoin(InetAddress endpoint, EndpointState epState) {}
- public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
- public void onAlive(InetAddress endpoint, EndpointState state) {}
- public void onDead(InetAddress endpoint, EndpointState state) {}
-
- public void onRemove(InetAddress endpoint)
- {
- convict(endpoint, Double.MAX_VALUE);
- }
-
- public void onRestart(InetAddress endpoint, EndpointState epState)
- {
- convict(endpoint, Double.MAX_VALUE);
- }
-
- public void convict(InetAddress endpoint, double phi)
- {
- if (!endpoint.equals(getHost()))
- return;
-
- // We want a higher confidence in the failure detection than usual because failing a streaming wrongly has a high cost.
- if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold())
- return;
-
- logger.error("Stream failed because {} died or was restarted/removed (streams may still be active "
- + "in background, but further streams won't be started)", endpoint);
- close(false);
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
new file mode 100644
index 0000000..27ea5af
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
@@ -0,0 +1,309 @@
+/*
+ * 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.streaming;
+
+import java.io.IOException;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.concurrent.PriorityBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.messages.StreamInitMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+/**
+ * ConnectionHandler manages incoming/outgoing message exchange for the {@link StreamSession}.
+ *
+ * <p>
+ * Internally, ConnectionHandler manages thread to receive incoming {@link StreamMessage} and thread to
+ * send outgoing message. Messages are encoded/decoded on those thread and handed to
+ * {@link StreamSession#messageReceived(org.apache.cassandra.streaming.messages.StreamMessage)}.
+ */
+public class ConnectionHandler
+{
+ private static final Logger logger = LoggerFactory.getLogger(ConnectionHandler.class);
+
+ private static final int MAX_CONNECT_ATTEMPTS = 3;
+
+ private final StreamSession session;
+ private final int protocolVersion;
+
+ private IncomingMessageHandler incoming;
+ private OutgoingMessageHandler outgoing;
+
+ private boolean connected = false;
+ private Socket socket;
+
+ ConnectionHandler(StreamSession session)
+ {
+ this.session = session;
+ this.protocolVersion = StreamMessage.CURRENT_VERSION;
+ }
+
+ ConnectionHandler(StreamSession session, Socket socket, int protocolVersion)
+ {
+ this.session = session;
+ this.socket = Preconditions.checkNotNull(socket);
+ this.connected = socket.isConnected();
+ this.protocolVersion = protocolVersion;
+ }
+
+ /**
+ * Connect to peer and start exchanging message.
+ * When connect attempt fails, this retries for maximum of MAX_CONNECT_ATTEMPTS times.
+ *
+ * @throws IOException when connection failed.
+ */
+ public void connect() throws IOException
+ {
+ int attempts = 0;
+ while (true)
+ {
+ try
+ {
+ socket = MessagingService.instance().getConnectionPool(session.peer).newSocket();
+ socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
+ break;
+ }
+ catch (IOException e)
+ {
+ if (++attempts >= MAX_CONNECT_ATTEMPTS)
+ throw e;
+
+ long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, attempts);
+ logger.warn("Failed attempt " + attempts + " to connect to " + session.peer + ". Retrying in " + waitms + " ms. (" + e + ")");
+ try
+ {
+ Thread.sleep(waitms);
+ }
+ catch (InterruptedException wtf)
+ {
+ throw new IOException("interrupted", wtf);
+ }
+ }
+ }
+ // send stream init message
+ SocketChannel channel = socket.getChannel();
+ WritableByteChannel out = channel;
+ // socket channel is null when encrypted(SSL)
+ if (channel == null)
+ {
+ out = Channels.newChannel(socket.getOutputStream());
+ }
+ logger.debug("Sending stream init...");
+ StreamInitMessage message = new StreamInitMessage(session.planId(), session.description());
+ out.write(message.createMessage(false, protocolVersion));
+
+ connected = true;
+
+ start();
+ session.onConnect();
+ }
+
+ public void close()
+ {
+ incoming.terminate();
+ outgoing.terminate();
+ if (socket != null && !isConnected())
+ {
+ try
+ {
+ socket.close();
+ }
+ catch (IOException ignore) {}
+ }
+ }
+
+ /**
+ * Start incoming/outgoing messaging threads.
+ */
+ public void start() throws IOException
+ {
+ SocketChannel channel = socket.getChannel();
+ ReadableByteChannel in = channel;
+ WritableByteChannel out = channel;
+ // socket channel is null when encrypted(SSL)
+ if (channel == null)
+ {
+ in = Channels.newChannel(socket.getInputStream());
+ out = Channels.newChannel(socket.getOutputStream());
+ }
+
+ incoming = new IncomingMessageHandler(session, protocolVersion, in);
+ outgoing = new OutgoingMessageHandler(session, protocolVersion, out);
+
+ // ready to send/receive files
+ new Thread(incoming, "STREAM-IN-" + session.peer).start();
+ new Thread(outgoing, "STREAM-OUT-" + session.peer).start();
+ }
+
+ public boolean isConnected()
+ {
+ return connected;
+ }
+
+ /**
+ * Enqueue messages to be sent.
+ *
+ * @param messages messages to send
+ */
+ public void sendMessages(Collection<? extends StreamMessage> messages)
+ {
+ for (StreamMessage message : messages)
+ sendMessage(message);
+ }
+
+ public void sendMessage(StreamMessage message)
+ {
+ assert isConnected();
+ outgoing.enqueue(message);
+ }
+
+ abstract static class MessageHandler implements Runnable
+ {
+ protected final StreamSession session;
+ protected final int protocolVersion;
+ private volatile boolean terminated;
+
+ protected MessageHandler(StreamSession session, int protocolVersion)
+ {
+ this.session = session;
+ this.protocolVersion = protocolVersion;
+ }
+
+ public void terminate()
+ {
+ terminated = true;
+ }
+
+ public boolean terminated()
+ {
+ return terminated;
+ }
+ }
+
+ /**
+ * Incoming streaming message handler
+ */
+ static class IncomingMessageHandler extends MessageHandler
+ {
+ private final ReadableByteChannel in;
+
+ IncomingMessageHandler(StreamSession session, int protocolVersion, ReadableByteChannel in)
+ {
+ super(session, protocolVersion);
+ this.in = in;
+ }
+
+ public void run()
+ {
+ while (!terminated())
+ {
+ try
+ {
+ // receive message
+ StreamMessage message = StreamMessage.deserialize(in, protocolVersion, session);
+ assert message != null;
+ session.messageReceived(message);
+ }
+ catch (SocketException e)
+ {
+ // socket is closed
+ terminate();
+ }
+ catch (Throwable e)
+ {
+ session.onError(e);
+ }
+ }
+ }
+ }
+
+ /**
+ * Outgoing file transfer thread
+ */
+ static class OutgoingMessageHandler extends MessageHandler
+ {
+ /*
+ * All out going messages are queued up into messageQueue.
+ * The size will grow when received streaming request.
+ *
+ * Queue is also PriorityQueue so that prior messages can go out fast.
+ */
+ private final PriorityBlockingQueue<StreamMessage> messageQueue = new PriorityBlockingQueue<>(64, new Comparator<StreamMessage>()
+ {
+ public int compare(StreamMessage o1, StreamMessage o2)
+ {
+ return o2.getPriority() - o1.getPriority();
+ }
+ });
+
+ private final WritableByteChannel out;
+
+ OutgoingMessageHandler(StreamSession session, int protocolVersion, WritableByteChannel out)
+ {
+ super(session, protocolVersion);
+ this.out = out;
+ }
+
+ public void enqueue(StreamMessage message)
+ {
+ messageQueue.put(message);
+ }
+
+ public void run()
+ {
+ while (!terminated())
+ {
+ try
+ {
+ StreamMessage next = messageQueue.poll(1, TimeUnit.SECONDS);
+ if (next != null)
+ {
+ logger.debug("Sending " + next);
+ StreamMessage.serialize(next, out, protocolVersion, session);
+ if (next.type == StreamMessage.Type.SESSION_FAILED)
+ terminate();
+ }
+ }
+ catch (SocketException e)
+ {
+ session.onError(e);
+ terminate();
+ }
+ catch (InterruptedException | IOException e)
+ {
+ session.onError(e);
+ }
+ }
+ }
+ }
+}