You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2014/09/20 00:13:37 UTC
[1/6] git commit: Fix NPE when table dropped during streaming
Repository: cassandra
Updated Branches:
refs/heads/cassandra-2.0 6198a7563 -> d143487cb
refs/heads/cassandra-2.1 8589d0473 -> 7e49ed3e0
refs/heads/trunk 0ef0394df -> 2a59f815e
Fix NPE when table dropped during streaming
patch by yukim; reviewed by krummas for CASSANDRA-7946
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d143487c
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d143487c
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d143487c
Branch: refs/heads/cassandra-2.0
Commit: d143487cb198051d0eaccc7f587f35cc63fc85a9
Parents: 6198a75
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Sep 19 17:11:25 2014 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Sep 19 17:11:25 2014 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/streaming/StreamReader.java | 5 +++++
.../org/apache/cassandra/streaming/StreamReceiveTask.java | 8 ++++++++
.../cassandra/streaming/compress/CompressedStreamReader.java | 5 +++++
4 files changed, 19 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index abd7c68..fd49b09 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -20,6 +20,7 @@
* Always send Paxos commit to all replicas (CASSANDRA-7479)
* Make disruptor_thrift_server invocation pool configurable (CASSANDRA-7594)
* Make repair no-op when RF=1 (CASSANDRA-7864)
+ * Fix NPE when table dropped during streaming (CASSANDRA-7946)
Merged from 1.2:
* Don't index tombstones (CASSANDRA-7828)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 15aa3cb..3b2a924 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -75,6 +75,11 @@ public class StreamReader
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
SSTableWriter writer = createWriter(cfs, totalSize);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 223a46e..33da3d1 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -103,6 +103,14 @@ public class StreamReceiveTask extends StreamTask
public void run()
{
Pair<String, String> kscf = Schema.instance.getCF(task.cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ for (SSTableWriter writer : task.sstables)
+ writer.abort();
+ task.sstables.clear();
+ return;
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
StreamLockfile lockfile = new StreamLockfile(cfs.directories.getWriteableLocationAsFile(), UUID.randomUUID());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 4aac941..219cabb 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -60,6 +60,11 @@ public class CompressedStreamReader extends StreamReader
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
SSTableWriter writer = createWriter(cfs, totalSize);
[3/6] git commit: Fix NPE when table dropped during streaming
Posted by yu...@apache.org.
Fix NPE when table dropped during streaming
patch by yukim; reviewed by krummas for CASSANDRA-7946
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d143487c
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d143487c
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d143487c
Branch: refs/heads/trunk
Commit: d143487cb198051d0eaccc7f587f35cc63fc85a9
Parents: 6198a75
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Sep 19 17:11:25 2014 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Sep 19 17:11:25 2014 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/streaming/StreamReader.java | 5 +++++
.../org/apache/cassandra/streaming/StreamReceiveTask.java | 8 ++++++++
.../cassandra/streaming/compress/CompressedStreamReader.java | 5 +++++
4 files changed, 19 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index abd7c68..fd49b09 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -20,6 +20,7 @@
* Always send Paxos commit to all replicas (CASSANDRA-7479)
* Make disruptor_thrift_server invocation pool configurable (CASSANDRA-7594)
* Make repair no-op when RF=1 (CASSANDRA-7864)
+ * Fix NPE when table dropped during streaming (CASSANDRA-7946)
Merged from 1.2:
* Don't index tombstones (CASSANDRA-7828)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 15aa3cb..3b2a924 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -75,6 +75,11 @@ public class StreamReader
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
SSTableWriter writer = createWriter(cfs, totalSize);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 223a46e..33da3d1 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -103,6 +103,14 @@ public class StreamReceiveTask extends StreamTask
public void run()
{
Pair<String, String> kscf = Schema.instance.getCF(task.cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ for (SSTableWriter writer : task.sstables)
+ writer.abort();
+ task.sstables.clear();
+ return;
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
StreamLockfile lockfile = new StreamLockfile(cfs.directories.getWriteableLocationAsFile(), UUID.randomUUID());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 4aac941..219cabb 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -60,6 +60,11 @@ public class CompressedStreamReader extends StreamReader
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
SSTableWriter writer = createWriter(cfs, totalSize);
[5/6] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Posted by yu...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1
Conflicts:
CHANGES.txt
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7e49ed3e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7e49ed3e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7e49ed3e
Branch: refs/heads/cassandra-2.1
Commit: 7e49ed3e060992435bd0eb726cfd263e9fcee446
Parents: 8589d04 d143487
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Sep 19 17:13:16 2014 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Sep 19 17:13:16 2014 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/streaming/StreamReader.java | 5 +++++
.../org/apache/cassandra/streaming/StreamReceiveTask.java | 8 ++++++++
.../cassandra/streaming/compress/CompressedStreamReader.java | 5 +++++
4 files changed, 19 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7e49ed3e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ee313a8,fd49b09..7384829
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -57,53 -10,9 +57,54 @@@ Merged from 2.0
* Fix RowIndexEntry to report correct serializedSize (CASSANDRA-7948)
* Make CQLSSTableWriter sync within partitions (CASSANDRA-7360)
* Potentially use non-local replicas in CqlConfigHelper (CASSANDRA-7906)
- * Explicitly disallowing mixing multi-column and single-column
+ * Explicitly disallow mixing multi-column and single-column
relations on clustering columns (CASSANDRA-7711)
* Better error message when condition is set on PK column (CASSANDRA-7804)
+ * Don't send schema change responses and events for no-op DDL
+ statements (CASSANDRA-7600)
+ * (Hadoop) fix cluster initialisation for a split fetching (CASSANDRA-7774)
+ * Throw InvalidRequestException when queries contain relations on entire
+ collection columns (CASSANDRA-7506)
+ * (cqlsh) enable CTRL-R history search with libedit (CASSANDRA-7577)
+ * (Hadoop) allow ACFRW to limit nodes to local DC (CASSANDRA-7252)
+ * (cqlsh) cqlsh should automatically disable tracing when selecting
+ from system_traces (CASSANDRA-7641)
+ * (Hadoop) Add CqlOutputFormat (CASSANDRA-6927)
+ * Don't depend on cassandra config for nodetool ring (CASSANDRA-7508)
+ * (cqlsh) Fix failing cqlsh formatting tests (CASSANDRA-7703)
+ * Fix IncompatibleClassChangeError from hadoop2 (CASSANDRA-7229)
+ * Add 'nodetool sethintedhandoffthrottlekb' (CASSANDRA-7635)
+ * (cqlsh) Add tab-completion for CREATE/DROP USER IF [NOT] EXISTS (CASSANDRA-7611)
+ * Catch errors when the JVM pulls the rug out from GCInspector (CASSANDRA-5345)
+ * cqlsh fails when version number parts are not int (CASSANDRA-7524)
++ * Fix NPE when table dropped during streaming (CASSANDRA-7946)
+Merged from 1.2:
+ * Don't index tombstones (CASSANDRA-7828)
+ * Improve PasswordAuthenticator default super user setup (CASSANDRA-7788)
+
+
+2.1.0
+ * (cqlsh) Removed "ALTER TYPE <name> RENAME TO <name>" from tab-completion
+ (CASSANDRA-7895)
+ * Fixed IllegalStateException in anticompaction (CASSANDRA-7892)
+ * cqlsh: DESCRIBE support for frozen UDTs, tuples (CASSANDRA-7863)
+ * Avoid exposing internal classes over JMX (CASSANDRA-7879)
+ * Add null check for keys when freezing collection (CASSANDRA-7869)
+ * Improve stress workload realism (CASSANDRA-7519)
+
+
+2.1.0-rc7
+ * Add frozen keyword and require UDT to be frozen (CASSANDRA-7857)
+ * Track added sstable size correctly (CASSANDRA-7239)
+ * (cqlsh) Fix case insensitivity (CASSANDRA-7834)
+ * Fix failure to stream ranges when moving (CASSANDRA-7836)
+ * Correctly remove tmplink files (CASSANDRA-7803)
+ * (cqlsh) Fix column name formatting for functions, CAS operations,
+ and UDT field selections (CASSANDRA-7806)
+ * (cqlsh) Fix COPY FROM handling of null/empty primary key
+ values (CASSANDRA-7792)
+ * Fix ordering of static cells (CASSANDRA-7763)
+Merged from 2.0:
* Forbid re-adding dropped counter columns (CASSANDRA-7831)
* Fix CFMetaData#isThriftCompatible() for PK-only tables (CASSANDRA-7832)
* Always reject inequality on the partition key without token()
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7e49ed3e/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamReader.java
index f47479f,3b2a924..3014549
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@@ -83,9 -75,14 +83,14 @@@ public class StreamReade
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
- SSTableWriter writer = createWriter(cfs, totalSize);
+ SSTableWriter writer = createWriter(cfs, totalSize, repairedAt);
DataInputStream dis = new DataInputStream(new LZFInputStream(Channels.newInputStream(channel)));
BytesReadTracker in = new BytesReadTracker(dis);
try
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7e49ed3e/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index ebd191b,219cabb..fb2599f
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@@ -65,9 -60,14 +65,14 @@@ public class CompressedStreamReader ext
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
- SSTableWriter writer = createWriter(cfs, totalSize);
+ SSTableWriter writer = createWriter(cfs, totalSize, repairedAt);
CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo, inputVersion.hasPostCompressionAdlerChecksums);
BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));
[2/6] git commit: Fix NPE when table dropped during streaming
Posted by yu...@apache.org.
Fix NPE when table dropped during streaming
patch by yukim; reviewed by krummas for CASSANDRA-7946
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d143487c
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d143487c
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d143487c
Branch: refs/heads/cassandra-2.1
Commit: d143487cb198051d0eaccc7f587f35cc63fc85a9
Parents: 6198a75
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Sep 19 17:11:25 2014 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Sep 19 17:11:25 2014 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/streaming/StreamReader.java | 5 +++++
.../org/apache/cassandra/streaming/StreamReceiveTask.java | 8 ++++++++
.../cassandra/streaming/compress/CompressedStreamReader.java | 5 +++++
4 files changed, 19 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index abd7c68..fd49b09 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -20,6 +20,7 @@
* Always send Paxos commit to all replicas (CASSANDRA-7479)
* Make disruptor_thrift_server invocation pool configurable (CASSANDRA-7594)
* Make repair no-op when RF=1 (CASSANDRA-7864)
+ * Fix NPE when table dropped during streaming (CASSANDRA-7946)
Merged from 1.2:
* Don't index tombstones (CASSANDRA-7828)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 15aa3cb..3b2a924 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -75,6 +75,11 @@ public class StreamReader
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
SSTableWriter writer = createWriter(cfs, totalSize);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 223a46e..33da3d1 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -103,6 +103,14 @@ public class StreamReceiveTask extends StreamTask
public void run()
{
Pair<String, String> kscf = Schema.instance.getCF(task.cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ for (SSTableWriter writer : task.sstables)
+ writer.abort();
+ task.sstables.clear();
+ return;
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
StreamLockfile lockfile = new StreamLockfile(cfs.directories.getWriteableLocationAsFile(), UUID.randomUUID());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d143487c/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
index 4aac941..219cabb 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -60,6 +60,11 @@ public class CompressedStreamReader extends StreamReader
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
SSTableWriter writer = createWriter(cfs, totalSize);
[6/6] git commit: Merge branch 'cassandra-2.1' into trunk
Posted by yu...@apache.org.
Merge branch 'cassandra-2.1' into trunk
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2a59f815
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2a59f815
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2a59f815
Branch: refs/heads/trunk
Commit: 2a59f815e4380ce713484b676ecb72a4778aa4ed
Parents: 0ef0394 7e49ed3
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Sep 19 17:13:23 2014 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Sep 19 17:13:23 2014 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/streaming/StreamReader.java | 5 +++++
.../org/apache/cassandra/streaming/StreamReceiveTask.java | 8 ++++++++
.../cassandra/streaming/compress/CompressedStreamReader.java | 5 +++++
4 files changed, 19 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a59f815/CHANGES.txt
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a59f815/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
[4/6] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Posted by yu...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1
Conflicts:
CHANGES.txt
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7e49ed3e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7e49ed3e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7e49ed3e
Branch: refs/heads/trunk
Commit: 7e49ed3e060992435bd0eb726cfd263e9fcee446
Parents: 8589d04 d143487
Author: Yuki Morishita <yu...@apache.org>
Authored: Fri Sep 19 17:13:16 2014 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Fri Sep 19 17:13:16 2014 -0500
----------------------------------------------------------------------
CHANGES.txt | 1 +
src/java/org/apache/cassandra/streaming/StreamReader.java | 5 +++++
.../org/apache/cassandra/streaming/StreamReceiveTask.java | 8 ++++++++
.../cassandra/streaming/compress/CompressedStreamReader.java | 5 +++++
4 files changed, 19 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7e49ed3e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index ee313a8,fd49b09..7384829
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -57,53 -10,9 +57,54 @@@ Merged from 2.0
* Fix RowIndexEntry to report correct serializedSize (CASSANDRA-7948)
* Make CQLSSTableWriter sync within partitions (CASSANDRA-7360)
* Potentially use non-local replicas in CqlConfigHelper (CASSANDRA-7906)
- * Explicitly disallowing mixing multi-column and single-column
+ * Explicitly disallow mixing multi-column and single-column
relations on clustering columns (CASSANDRA-7711)
* Better error message when condition is set on PK column (CASSANDRA-7804)
+ * Don't send schema change responses and events for no-op DDL
+ statements (CASSANDRA-7600)
+ * (Hadoop) fix cluster initialisation for a split fetching (CASSANDRA-7774)
+ * Throw InvalidRequestException when queries contain relations on entire
+ collection columns (CASSANDRA-7506)
+ * (cqlsh) enable CTRL-R history search with libedit (CASSANDRA-7577)
+ * (Hadoop) allow ACFRW to limit nodes to local DC (CASSANDRA-7252)
+ * (cqlsh) cqlsh should automatically disable tracing when selecting
+ from system_traces (CASSANDRA-7641)
+ * (Hadoop) Add CqlOutputFormat (CASSANDRA-6927)
+ * Don't depend on cassandra config for nodetool ring (CASSANDRA-7508)
+ * (cqlsh) Fix failing cqlsh formatting tests (CASSANDRA-7703)
+ * Fix IncompatibleClassChangeError from hadoop2 (CASSANDRA-7229)
+ * Add 'nodetool sethintedhandoffthrottlekb' (CASSANDRA-7635)
+ * (cqlsh) Add tab-completion for CREATE/DROP USER IF [NOT] EXISTS (CASSANDRA-7611)
+ * Catch errors when the JVM pulls the rug out from GCInspector (CASSANDRA-5345)
+ * cqlsh fails when version number parts are not int (CASSANDRA-7524)
++ * Fix NPE when table dropped during streaming (CASSANDRA-7946)
+Merged from 1.2:
+ * Don't index tombstones (CASSANDRA-7828)
+ * Improve PasswordAuthenticator default super user setup (CASSANDRA-7788)
+
+
+2.1.0
+ * (cqlsh) Removed "ALTER TYPE <name> RENAME TO <name>" from tab-completion
+ (CASSANDRA-7895)
+ * Fixed IllegalStateException in anticompaction (CASSANDRA-7892)
+ * cqlsh: DESCRIBE support for frozen UDTs, tuples (CASSANDRA-7863)
+ * Avoid exposing internal classes over JMX (CASSANDRA-7879)
+ * Add null check for keys when freezing collection (CASSANDRA-7869)
+ * Improve stress workload realism (CASSANDRA-7519)
+
+
+2.1.0-rc7
+ * Add frozen keyword and require UDT to be frozen (CASSANDRA-7857)
+ * Track added sstable size correctly (CASSANDRA-7239)
+ * (cqlsh) Fix case insensitivity (CASSANDRA-7834)
+ * Fix failure to stream ranges when moving (CASSANDRA-7836)
+ * Correctly remove tmplink files (CASSANDRA-7803)
+ * (cqlsh) Fix column name formatting for functions, CAS operations,
+ and UDT field selections (CASSANDRA-7806)
+ * (cqlsh) Fix COPY FROM handling of null/empty primary key
+ values (CASSANDRA-7792)
+ * Fix ordering of static cells (CASSANDRA-7763)
+Merged from 2.0:
* Forbid re-adding dropped counter columns (CASSANDRA-7831)
* Fix CFMetaData#isThriftCompatible() for PK-only tables (CASSANDRA-7832)
* Always reject inequality on the partition key without token()
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7e49ed3e/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamReader.java
index f47479f,3b2a924..3014549
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@@ -83,9 -75,14 +83,14 @@@ public class StreamReade
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
- SSTableWriter writer = createWriter(cfs, totalSize);
+ SSTableWriter writer = createWriter(cfs, totalSize, repairedAt);
DataInputStream dis = new DataInputStream(new LZFInputStream(Channels.newInputStream(channel)));
BytesReadTracker in = new BytesReadTracker(dis);
try
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7e49ed3e/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index ebd191b,219cabb..fb2599f
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@@ -65,9 -60,14 +65,14 @@@ public class CompressedStreamReader ext
long totalSize = totalSize();
Pair<String, String> kscf = Schema.instance.getCF(cfId);
+ if (kscf == null)
+ {
+ // schema was dropped during streaming
+ throw new IOException("CF " + cfId + " was dropped during streaming");
+ }
ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
- SSTableWriter writer = createWriter(cfs, totalSize);
+ SSTableWriter writer = createWriter(cfs, totalSize, repairedAt);
CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo, inputVersion.hasPostCompressionAdlerChecksums);
BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));