Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7C0BC19AEF for ; Tue, 29 Mar 2016 09:23:18 +0000 (UTC) Received: (qmail 15272 invoked by uid 500); 29 Mar 2016 09:23:17 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 15235 invoked by uid 500); 29 Mar 2016 09:23:17 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 15013 invoked by uid 99); 29 Mar 2016 09:23:17 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Mar 2016 09:23:17 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E0095E9812; Tue, 29 Mar 2016 09:23:16 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: marcuse@apache.org To: commits@cassandra.apache.org Date: Tue, 29 Mar 2016 09:23:21 -0000 Message-Id: <6eca35f03ed740caa1d38d8b0b44d54e@git.apache.org> In-Reply-To: <4bccbdaacc35459da06d481f873e0d92@git.apache.org> References: <4bccbdaacc35459da06d481f873e0d92@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [06/10] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0 Merge branch 'cassandra-2.2' into cassandra-3.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/01048346 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/01048346 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/01048346 Branch: refs/heads/cassandra-3.0 Commit: 01048346e436cacfd28a38447469bfd35e689883 Parents: 0c91977 aefea62 Author: Marcus Eriksson Authored: Tue Mar 29 11:20:55 2016 +0200 Committer: Marcus Eriksson Committed: Tue Mar 29 11:20:55 2016 +0200 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../org/apache/cassandra/db/lifecycle/LifecycleTransaction.java | 5 ++++- .../org/apache/cassandra/io/sstable/SSTableWriterTestBase.java | 3 +++ 3 files changed, 8 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/01048346/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index fdc873e,5e8828b..a9ccf98 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,16 -1,5 +1,17 @@@ -2.2.6 +3.0.5 + * Enable SO_REUSEADDR for JMX RMI server sockets (CASSANDRA-11093) + * Allocate merkletrees with the correct size (CASSANDRA-11390) + * Support streaming pre-3.0 sstables (CASSANDRA-10990) + * Add backpressure to compressed commit log (CASSANDRA-10971) + * SSTableExport supports secondary index tables (CASSANDRA-11330) + * Fix sstabledump to include missing info in debug output (CASSANDRA-11321) + * Establish and implement canonical bulk reading workload(s) (CASSANDRA-10331) + * Fix paging for IN queries on tables without clustering columns (CASSANDRA-11208) + * Remove recursive call from CompositesSearcher (CASSANDRA-11304) + * Fix filtering on non-primary key columns for queries without index (CASSANDRA-6377) + * Fix sstableloader fail when using materialized view (CASSANDRA-11275) +Merged from 2.2: + * Notify when sstables change after cancelling compaction (CASSANDRA-11373) * cqlsh: COPY FROM should check that explicit column names are valid (CASSANDRA-11333) * Add -Dcassandra.start_gossip startup option (CASSANDRA-10809) * Fix UTF8Validator.validate() for modified UTF-8 (CASSANDRA-10748) http://git-wip-us.apache.org/repos/asf/cassandra/blob/01048346/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java index a5eb01f,59cee50..91515aa --- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java +++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java @@@ -236,15 -176,15 +236,18 @@@ public class LifecycleTransaction exten // mark obsolete all readers that are not versions of those present in the original set Iterable obsolete = filterOut(concatUniq(staged.update, logged.update), originals); logger.trace("Obsoleting {}", obsolete); - // we don't pass the tracker in for the obsoletion, since these readers have never been notified externally - // nor had their size accounting affected - accumulate = markObsolete(null, obsolete, accumulate); + + accumulate = prepareForObsoletion(obsolete, log, obsoletions = new ArrayList<>(), accumulate); + // it's safe to abort even if committed, see maybeFail in doCommit() above, in this case it will just report + // a failure to abort, which is useful information to have for debug + accumulate = log.abort(accumulate); + accumulate = markObsolete(obsoletions, accumulate); // replace all updated readers with a version restored to its original state - accumulate = tracker.apply(updateLiveSet(logged.update, restoreUpdatedOriginals()), accumulate); + List restored = restoreUpdatedOriginals(); + List invalid = Lists.newArrayList(Iterables.concat(logged.update, logged.obsolete)); + accumulate = tracker.apply(updateLiveSet(logged.update, restored), accumulate); + accumulate = tracker.notifySSTablesChanged(invalid, restored, OperationType.COMPACTION, accumulate); // setReplaced immediately preceding versions that have not been obsoleted accumulate = setReplaced(logged.update, accumulate); // we have replaced all of logged.update and never made visible staged.update, http://git-wip-us.apache.org/repos/asf/cassandra/blob/01048346/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java ---------------------------------------------------------------------- diff --cc test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java index 0af743d,0000000..2db92f7 mode 100644,000000..100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java @@@ -1,166 -1,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.io.sstable; + +import java.io.File; +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.SerializationHeader; ++import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.utils.FBUtilities; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class SSTableWriterTestBase extends SchemaLoader +{ + + protected static final String KEYSPACE = "SSTableRewriterTest"; + protected static final String CF = "Standard1"; + + private static Config.DiskAccessMode standardMode; + private static Config.DiskAccessMode indexMode; + + @BeforeClass + public static void defineSchema() throws ConfigurationException + { + if (FBUtilities.isWindows()) + { + standardMode = DatabaseDescriptor.getDiskAccessMode(); + indexMode = DatabaseDescriptor.getIndexAccessMode(); + + DatabaseDescriptor.setDiskAccessMode(Config.DiskAccessMode.standard); + DatabaseDescriptor.setIndexAccessMode(Config.DiskAccessMode.standard); + } + + SchemaLoader.prepareServer(); + SchemaLoader.createKeyspace(KEYSPACE, + KeyspaceParams.simple(1), + SchemaLoader.standardCFMD(KEYSPACE, CF)); + } + + @AfterClass + public static void revertDiskAccess() + { + DatabaseDescriptor.setDiskAccessMode(standardMode); + DatabaseDescriptor.setIndexAccessMode(indexMode); + } + + @After + public void truncateCF() + { + Keyspace keyspace = Keyspace.open(KEYSPACE); + ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF); + store.truncateBlocking(); + LifecycleTransaction.waitForDeletions(); + } + + public static void truncate(ColumnFamilyStore cfs) + { + cfs.truncateBlocking(); + LifecycleTransaction.waitForDeletions(); + Uninterruptibles.sleepUninterruptibly(10L, TimeUnit.MILLISECONDS); + assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount()); + assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount()); + validateCFS(cfs); + } + + public static void validateCFS(ColumnFamilyStore cfs) + { + Set liveDescriptors = new HashSet<>(); + long spaceUsed = 0; + for (SSTableReader sstable : cfs.getLiveSSTables()) + { + assertFalse(sstable.isMarkedCompacted()); + assertEquals(1, sstable.selfRef().globalCount()); + liveDescriptors.add(sstable.descriptor.generation); + spaceUsed += sstable.bytesOnDisk(); + } + for (File dir : cfs.getDirectories().getCFDirectories()) + { + for (File f : dir.listFiles()) + { + if (f.getName().contains("Data")) + { + Descriptor d = Descriptor.fromFilename(f.getAbsolutePath()); + assertTrue(d.toString(), liveDescriptors.contains(d.generation)); + } + } + } + assertEquals(spaceUsed, cfs.metric.liveDiskSpaceUsed.getCount()); + assertEquals(spaceUsed, cfs.metric.totalDiskSpaceUsed.getCount()); + assertTrue(cfs.getTracker().getCompacting().isEmpty()); ++ if(cfs.getLiveSSTables().size() > 0) ++ assertFalse(CompactionManager.instance.submitMaximal(cfs, cfs.gcBefore((int) (System.currentTimeMillis() / 1000)), false).isEmpty()); + } + + public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn) + { + String filename = cfs.getSSTablePath(directory); + return SSTableWriter.create(filename, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS), txn); + } + + public static ByteBuffer random(int i, int size) + { + byte[] bytes = new byte[size + 4]; + ThreadLocalRandom.current().nextBytes(bytes); + ByteBuffer r = ByteBuffer.wrap(bytes); + r.putInt(0, i); + return r; + } + + public static int assertFileCounts(String [] files) + { + int tmplinkcount = 0; + int tmpcount = 0; + int datacount = 0; + for (String f : files) + { + if (f.endsWith("-CRC.db")) + continue; + if (f.contains("tmplink-")) + tmplinkcount++; + else if (f.contains("tmp-")) + tmpcount++; + else if (f.contains("Data")) + datacount++; + } + assertEquals(0, tmplinkcount); + assertEquals(0, tmpcount); + return datacount; + } +}