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 2015/05/26 17:51:32 UTC

[1/3] cassandra git commit: Let CassandraVersion handle SNAPSHOT version

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 3adfd1575 -> 2385dc2d9
  refs/heads/trunk 2ec7594fd -> 13409fdf0


Let CassandraVersion handle SNAPSHOT version

patch by yukim; reviewed by Jeremiah Jordan for CASSANDRA-9438


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2385dc2d
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2385dc2d
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2385dc2d

Branch: refs/heads/cassandra-2.2
Commit: 2385dc2d912391190d4f834bd43f6c1fdf368e4d
Parents: 3adfd15
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue May 26 10:50:41 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue May 26 10:50:41 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  10 +-
 .../cassandra/repair/AnticompactionTask.java    |   6 +-
 .../apache/cassandra/service/ClientState.java   |   8 +-
 .../transport/messages/StartupMessage.java      |   4 +-
 .../cassandra/utils/CassandraVersion.java       | 231 +++++++++++++++++++
 .../apache/cassandra/utils/SemanticVersion.java | 231 -------------------
 .../apache/cassandra/db/SystemKeyspaceTest.java |   4 +-
 .../cassandra/utils/CassandraVersionTest.java   | 146 ++++++++++++
 .../cassandra/utils/SemanticVersionTest.java    | 113 ---------
 11 files changed, 395 insertions(+), 361 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d4a8150..2a242c1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,7 @@
  * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
  * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)
  * Add ability to stop compaction by ID (CASSANDRA-7207)
+ * Let CassandraVersion handle SNAPSHOT version (CASSANDRA-9438)
 Merged from 2.1:
  * (cqlsh) Better float precision by default (CASSANDRA-9224)
  * Improve estimated row count (CASSANDRA-9107)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 2698a8f..7b9261c 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -56,7 +56,7 @@ import org.github.jamm.MemoryMeter;
 
 public class QueryProcessor implements QueryHandler
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.2.0");
+    public static final CassandraVersion CQL_VERSION = new CassandraVersion("3.2.0");
 
     public static final QueryProcessor instance = new QueryProcessor();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 6e754a4..67a3162 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -70,11 +70,11 @@ public final class SystemKeyspace
     // Used to indicate that there was a previous version written to the legacy (pre 1.2)
     // system.Versions table, but that we cannot read it. Suffice to say, any upgrade should
     // proceed through 1.2.x before upgrading to the current version.
-    public static final SemanticVersion UNREADABLE_VERSION = new SemanticVersion("0.0.0-unknown");
+    public static final CassandraVersion UNREADABLE_VERSION = new CassandraVersion("0.0.0-unknown");
 
     // Used to indicate that no previous version information was found. When encountered, we assume that
     // Cassandra was not previously installed and we're in the process of starting a fresh node.
-    public static final SemanticVersion NULL_VERSION = new SemanticVersion("0.0.0-absent");
+    public static final CassandraVersion NULL_VERSION = new CassandraVersion("0.0.0-absent");
 
     public static final String NAME = "system";
 
@@ -679,19 +679,19 @@ public final class SystemKeyspace
      * @param ep endpoint address to check
      * @return Release version or null if version is unknown.
      */
-    public static SemanticVersion getReleaseVersion(InetAddress ep)
+    public static CassandraVersion getReleaseVersion(InetAddress ep)
     {
         try
         {
             if (FBUtilities.getBroadcastAddress().equals(ep))
             {
-                return new SemanticVersion(FBUtilities.getReleaseVersionString());
+                return new CassandraVersion(FBUtilities.getReleaseVersionString());
             }
             String req = "SELECT release_version FROM system.%s WHERE peer=?";
             UntypedResultSet result = executeInternal(String.format(req, PEERS), ep);
             if (result != null && result.one().has("release_version"))
             {
-                return new SemanticVersion(result.one().getString("release_version"));
+                return new CassandraVersion(result.one().getString("release_version"));
             }
             // version is unknown
             return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/repair/AnticompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AnticompactionTask.java b/src/java/org/apache/cassandra/repair/AnticompactionTask.java
index d1bbb82..16de071 100644
--- a/src/java/org/apache/cassandra/repair/AnticompactionTask.java
+++ b/src/java/org/apache/cassandra/repair/AnticompactionTask.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.AnticompactionRequest;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 public class AnticompactionTask extends AbstractFuture<InetAddress> implements Runnable
 {
@@ -39,7 +39,7 @@ public class AnticompactionTask extends AbstractFuture<InetAddress> implements R
      * Version that anticompaction response is not supported up to.
      * If Cassandra version is more than this, we need to wait for anticompaction response.
      */
-    private static final SemanticVersion VERSION_CHECKER = new SemanticVersion("2.1.5");
+    private static final CassandraVersion VERSION_CHECKER = new CassandraVersion("2.1.5");
 
     private final UUID parentSession;
     private final InetAddress neighbor;
@@ -55,7 +55,7 @@ public class AnticompactionTask extends AbstractFuture<InetAddress> implements R
     public void run()
     {
         AnticompactionRequest acr = new AnticompactionRequest(parentSession, successfulRanges);
-        SemanticVersion peerVersion = SystemKeyspace.getReleaseVersion(neighbor);
+        CassandraVersion peerVersion = SystemKeyspace.getReleaseVersion(neighbor);
         if (peerVersion != null && peerVersion.compareTo(VERSION_CHECKER) > 0)
         {
             MessagingService.instance().sendRR(acr.createMessage(), neighbor, new AnticompactionCallback(this), TimeUnit.DAYS.toMillis(1), true);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index b171f08..b9e89f5 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -43,7 +43,7 @@ import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * State related to a client connection.
@@ -51,7 +51,7 @@ import org.apache.cassandra.utils.SemanticVersion;
 public class ClientState
 {
     private static final Logger logger = LoggerFactory.getLogger(ClientState.class);
-    public static final SemanticVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
+    public static final CassandraVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
 
     private static final Set<IResource> READABLE_SYSTEM_RESOURCES = new HashSet<>();
     private static final Set<IResource> PROTECTED_AUTH_RESOURCES = new HashSet<>();
@@ -352,9 +352,9 @@ public class ClientState
         return user;
     }
 
-    public static SemanticVersion[] getCQLSupportedVersion()
+    public static CassandraVersion[] getCQLSupportedVersion()
     {
-        return new SemanticVersion[]{ QueryProcessor.CQL_VERSION };
+        return new CassandraVersion[]{ QueryProcessor.CQL_VERSION };
     }
 
     private Set<Permission> authorize(IResource resource)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index ac7a1f2..04d8e62 100644
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@ -25,7 +25,7 @@ import io.netty.buffer.ByteBuf;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.*;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * The initial message of the protocol.
@@ -70,7 +70,7 @@ public class StartupMessage extends Message.Request
 
         try 
         {
-            if (new SemanticVersion(cqlVersion).compareTo(new SemanticVersion("2.99.0")) < 0)
+            if (new CassandraVersion(cqlVersion).compareTo(new CassandraVersion("2.99.0")) < 0)
                 throw new ProtocolException(String.format("CQL version %s is not supported by the binary protocol (supported version are >= 3.0.0)", cqlVersion));
         }
         catch (IllegalArgumentException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/utils/CassandraVersion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CassandraVersion.java b/src/java/org/apache/cassandra/utils/CassandraVersion.java
new file mode 100644
index 0000000..62d68be
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/CassandraVersion.java
@@ -0,0 +1,231 @@
+/*
+ * 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.utils;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Objects;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Implements versioning used in Cassandra and CQL.
+ * <p/>
+ * Note: The following code uses a slight variation from the semver document (http://semver.org).
+ */
+public class CassandraVersion implements Comparable<CassandraVersion>
+{
+    private static final String VERSION_REGEXP = "(\\d+)\\.(\\d+)\\.(\\d+)(\\-[.\\w]+)?([.+][.\\w]+)?";
+    private static final Pattern pattern = Pattern.compile(VERSION_REGEXP);
+    private static final Pattern SNAPSHOT = Pattern.compile("-SNAPSHOT");
+
+    public final int major;
+    public final int minor;
+    public final int patch;
+
+    private final String[] preRelease;
+    private final String[] build;
+
+    private CassandraVersion(int major, int minor, int patch, String[] preRelease, String[] build)
+    {
+        this.major = major;
+        this.minor = minor;
+        this.patch = patch;
+        this.preRelease = preRelease;
+        this.build = build;
+    }
+
+    /**
+     * Parse a version from a string.
+     *
+     * @param version the string to parse
+     * @throws IllegalArgumentException if the provided string does not
+     *                                  represent a version
+     */
+    public CassandraVersion(String version)
+    {
+        String stripped = SNAPSHOT.matcher(version).replaceFirst("");
+        Matcher matcher = pattern.matcher(stripped);
+        if (!matcher.matches())
+            throw new IllegalArgumentException("Invalid version value: " + version);
+
+        try
+        {
+            this.major = Integer.parseInt(matcher.group(1));
+            this.minor = Integer.parseInt(matcher.group(2));
+            this.patch = Integer.parseInt(matcher.group(3));
+
+            String pr = matcher.group(4);
+            String bld = matcher.group(5);
+
+            this.preRelease = pr == null || pr.isEmpty() ? null : parseIdentifiers(stripped, pr);
+            this.build = bld == null || bld.isEmpty() ? null : parseIdentifiers(stripped, bld);
+        }
+        catch (NumberFormatException e)
+        {
+            throw new IllegalArgumentException("Invalid version value: " + version);
+        }
+    }
+
+    private static String[] parseIdentifiers(String version, String str)
+    {
+        // Drop initial - or +
+        str = str.substring(1);
+        String[] parts = str.split("\\.");
+        for (String part : parts)
+        {
+            if (!part.matches("\\w+"))
+                throw new IllegalArgumentException("Invalid version value: " + version);
+        }
+        return parts;
+    }
+
+    public int compareTo(CassandraVersion other)
+    {
+        if (major < other.major)
+            return -1;
+        if (major > other.major)
+            return 1;
+
+        if (minor < other.minor)
+            return -1;
+        if (minor > other.minor)
+            return 1;
+
+        if (patch < other.patch)
+            return -1;
+        if (patch > other.patch)
+            return 1;
+
+        int c = compareIdentifiers(preRelease, other.preRelease, 1);
+        if (c != 0)
+            return c;
+
+        return compareIdentifiers(build, other.build, -1);
+    }
+
+    /**
+     * Returns a version that is backward compatible with this version amongst a list
+     * of provided version, or null if none can be found.
+     * <p/>
+     * For instance:
+     * "2.0.0".findSupportingVersion("2.0.0", "3.0.0") == "2.0.0"
+     * "2.0.0".findSupportingVersion("2.1.3", "3.0.0") == "2.1.3"
+     * "2.0.0".findSupportingVersion("3.0.0") == null
+     * "2.0.3".findSupportingVersion("2.0.0") == "2.0.0"
+     * "2.1.0".findSupportingVersion("2.0.0") == null
+     */
+    public CassandraVersion findSupportingVersion(CassandraVersion... versions)
+    {
+        for (CassandraVersion version : versions)
+        {
+            if (isSupportedBy(version))
+                return version;
+        }
+        return null;
+    }
+
+    public boolean isSupportedBy(CassandraVersion version)
+    {
+        return major == version.major && this.compareTo(version) <= 0;
+    }
+
+    private static int compareIdentifiers(String[] ids1, String[] ids2, int defaultPred)
+    {
+        if (ids1 == null)
+            return ids2 == null ? 0 : defaultPred;
+        else if (ids2 == null)
+            return -defaultPred;
+
+        int min = Math.min(ids1.length, ids2.length);
+        for (int i = 0; i < min; i++)
+        {
+            Integer i1 = tryParseInt(ids1[i]);
+            Integer i2 = tryParseInt(ids2[i]);
+
+            if (i1 != null)
+            {
+                // integer have precedence
+                if (i2 == null || i1 < i2)
+                    return -1;
+                else if (i1 > i2)
+                    return 1;
+            }
+            else
+            {
+                // integer have precedence
+                if (i2 != null)
+                    return 1;
+
+                int c = ids1[i].compareTo(ids2[i]);
+                if (c != 0)
+                    return c;
+            }
+        }
+
+        if (ids1.length < ids2.length)
+            return -1;
+        if (ids1.length > ids2.length)
+            return 1;
+        return 0;
+    }
+
+    private static Integer tryParseInt(String str)
+    {
+        try
+        {
+            return Integer.valueOf(str);
+        }
+        catch (NumberFormatException e)
+        {
+            return null;
+        }
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof CassandraVersion))
+            return false;
+        CassandraVersion that = (CassandraVersion) o;
+        return major == that.major
+               && minor == that.minor
+               && patch == that.patch
+               && Arrays.equals(preRelease, that.preRelease)
+               && Arrays.equals(build, that.build);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(major, minor, patch, preRelease, build);
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(major).append('.').append(minor).append('.').append(patch);
+        if (preRelease != null)
+            sb.append('-').append(StringUtils.join(preRelease, "."));
+        if (build != null)
+            sb.append('+').append(StringUtils.join(build, "."));
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/utils/SemanticVersion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/SemanticVersion.java b/src/java/org/apache/cassandra/utils/SemanticVersion.java
deleted file mode 100644
index 858029d..0000000
--- a/src/java/org/apache/cassandra/utils/SemanticVersion.java
+++ /dev/null
@@ -1,231 +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.utils;
-
-import java.util.regex.Pattern;
-import java.util.regex.Matcher;
-import java.util.Arrays;
-
-import org.apache.commons.lang3.StringUtils;
-import com.google.common.base.Objects;
-
-/**
- * Implements semantic versioning as defined at http://semver.org/.
- *
- * Note: The following code uses a slight variation from the document above in
- * that it doesn't allow dashes in pre-release and build identifier.
- */
-public class SemanticVersion implements Comparable<SemanticVersion>
-{
-    private static final String VERSION_REGEXP = "(\\d+)\\.(\\d+)\\.(\\d+)(\\-[.\\w]+)?([.+][.\\w]+)?";
-    private static final Pattern pattern = Pattern.compile(VERSION_REGEXP);
-
-    public final int major;
-    public final int minor;
-    public final int patch;
-
-    private final String[] preRelease;
-    private final String[] build;
-
-    private SemanticVersion(int major, int minor, int patch, String[] preRelease, String[] build)
-    {
-        this.major = major;
-        this.minor = minor;
-        this.patch = patch;
-        this.preRelease = preRelease;
-        this.build = build;
-    }
-
-    /**
-     * Parse a semantic version from a string.
-     *
-     * @param version the string to parse
-     * @throws IllegalArgumentException if the provided string does not
-     * represent a semantic version
-     */
-    public SemanticVersion(String version)
-    {
-        Matcher matcher = pattern.matcher(version);
-        if (!matcher.matches())
-            throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-
-        try
-        {
-            this.major = Integer.parseInt(matcher.group(1));
-            this.minor = Integer.parseInt(matcher.group(2));
-            this.patch = Integer.parseInt(matcher.group(3));
-
-            String pr = matcher.group(4);
-            String bld = matcher.group(5);
-
-            this.preRelease = pr == null || pr.isEmpty() ? null : parseIdentifiers(version, pr);
-            this.build = bld == null || bld.isEmpty() ? null : parseIdentifiers(version, bld);
-
-        }
-        catch (NumberFormatException e)
-        {
-            throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-        }
-    }
-
-    private static String[] parseIdentifiers(String version, String str)
-    {
-        // Drop initial - or +
-        str = str.substring(1);
-        String[] parts = str.split("\\.");
-        for (String part : parts)
-        {
-            if (!part.matches("\\w+"))
-                throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-        }
-        return parts;
-    }
-
-    public int compareTo(SemanticVersion other)
-    {
-        if (major < other.major)
-            return -1;
-        if (major > other.major)
-            return 1;
-
-        if (minor < other.minor)
-            return -1;
-        if (minor > other.minor)
-            return 1;
-
-        if (patch < other.patch)
-            return -1;
-        if (patch > other.patch)
-            return 1;
-
-        int c = compareIdentifiers(preRelease, other.preRelease, 1);
-        if (c != 0)
-            return c;
-
-        return compareIdentifiers(build, other.build, -1);
-    }
-
-    /**
-     * Returns a version that is backward compatible with this version amongst a list
-     * of provided version, or null if none can be found.
-     *
-     * For instance:
-     *   "2.0.0".findSupportingVersion("2.0.0", "3.0.0") == "2.0.0"
-     *   "2.0.0".findSupportingVersion("2.1.3", "3.0.0") == "2.1.3"
-     *   "2.0.0".findSupportingVersion("3.0.0") == null
-     *   "2.0.3".findSupportingVersion("2.0.0") == "2.0.0"
-     *   "2.1.0".findSupportingVersion("2.0.0") == null
-     */
-    public SemanticVersion findSupportingVersion(SemanticVersion... versions)
-    {
-        for (SemanticVersion version : versions)
-        {
-            if (isSupportedBy(version))
-                return version;
-        }
-        return null;
-    }
-
-    public boolean isSupportedBy(SemanticVersion version)
-    {
-        return major == version.major && this.compareTo(version) <= 0;
-    }
-
-    private static int compareIdentifiers(String[] ids1, String[] ids2, int defaultPred)
-    {
-        if (ids1 == null)
-            return ids2 == null ? 0 : defaultPred;
-        else if (ids2 == null)
-            return -defaultPred;
-
-        int min = Math.min(ids1.length, ids2.length);
-        for (int i = 0; i < min; i++)
-        {
-            Integer i1 = tryParseInt(ids1[i]);
-            Integer i2 = tryParseInt(ids2[i]);
-
-            if (i1 != null)
-            {
-                // integer have precedence
-                if (i2 == null || i1 < i2)
-                    return -1;
-                else if (i1 > i2)
-                    return 1;
-            }
-            else
-            {
-                // integer have precedence
-                if (i2 != null)
-                    return 1;
-
-                int c = ids1[i].compareTo(ids2[i]);
-                if (c != 0)
-                    return c;
-            }
-        }
-
-        if (ids1.length < ids2.length)
-            return -1;
-        if (ids1.length > ids2.length)
-            return 1;
-        return 0;
-    }
-
-    private static Integer tryParseInt(String str)
-    {
-        try
-        {
-            return Integer.valueOf(str);
-        }
-        catch (NumberFormatException e)
-        {
-            return null;
-        }
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if(!(o instanceof SemanticVersion))
-            return false;
-        SemanticVersion that = (SemanticVersion)o;
-        return major == that.major
-            && minor == that.minor
-            && patch == that.patch
-            && Arrays.equals(preRelease, that.preRelease)
-            && Arrays.equals(build, that.build);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Objects.hashCode(major, minor, patch, preRelease, build);
-    }
-
-    @Override
-    public String toString()
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append(major).append('.').append(minor).append('.').append(patch);
-        if (preRelease != null)
-            sb.append('-').append(StringUtils.join(preRelease, "."));
-        if (build != null)
-            sb.append('+').append(StringUtils.join(build, "."));
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index 7531b06..093f359 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -110,7 +110,7 @@ public class SystemKeyspaceTest
     private String getOlderVersionString()
     {
         String version = FBUtilities.getReleaseVersionString();
-        SemanticVersion semver = new SemanticVersion(version.contains("-") ? version.substring(0, version.indexOf('-'))
+        CassandraVersion semver = new CassandraVersion(version.contains("-") ? version.substring(0, version.indexOf('-'))
                                                                            : version);
         return (String.format("%s.%s.%s", semver.major - 1, semver.minor, semver.patch));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java b/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
new file mode 100644
index 0000000..145b735
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.utils;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+public class CassandraVersionTest
+{
+    @Test
+    public void testParsing()
+    {
+        CassandraVersion version;
+
+        version = new CassandraVersion("1.2.3");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+
+        version = new CassandraVersion("1.2.3-foo.2+Bar");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+
+        // CassandraVersion can parse 4th '.' as build number
+        version = new CassandraVersion("1.2.3.456");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+    }
+
+    @Test
+    public void testComparison()
+    {
+        CassandraVersion v1, v2;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.4");
+        assert v1.compareTo(v2) == -1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3");
+        assert v1.compareTo(v2) == 0;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("2.0.0");
+        assert v1.compareTo(v2) == -1;
+        assert v2.compareTo(v1) == 1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3-alpha");
+        assert v1.compareTo(v2) == 1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3+foo");
+        assert v1.compareTo(v2) == -1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3-alpha+foo");
+        assert v1.compareTo(v2) == 1;
+
+        v1 = new CassandraVersion("1.2.3-alpha+1");
+        v2 = new CassandraVersion("1.2.3-alpha+2");
+        assert v1.compareTo(v2) == -1;
+    }
+
+    @Test
+    public void testIsSupportedBy()
+    {
+        CassandraVersion v1, v2;
+
+        v1 = new CassandraVersion("3.0.2");
+        assert v1.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.4");
+        assert v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.3.3");
+        assert v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("2.2.3");
+        v2 = new CassandraVersion("1.3.3");
+        assert !v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("3.1.0");
+        v2 = new CassandraVersion("3.0.1");
+        assert !v1.isSupportedBy(v2);
+        assert v2.isSupportedBy(v1);
+    }
+
+    @Test
+    public void testInvalid()
+    {
+        assertThrows("1.0");
+        assertThrows("1.0.0a");
+        assertThrows("1.a.4");
+        assertThrows("1.0.0-foo&");
+    }
+
+    @Test
+    public void testSnapshot()
+    {
+        CassandraVersion prev, next;
+
+        prev = new CassandraVersion("2.1.5");
+        next = new CassandraVersion("2.1.5.123");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = next;
+        next = new CassandraVersion("2.2.0-beta1-SNAPSHOT");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = new CassandraVersion("2.2.0-beta1");
+        next = new CassandraVersion("2.2.0-rc1-SNAPSHOT");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = next;
+        next = new CassandraVersion("2.2.0");
+        assertTrue(prev.compareTo(next) < 0);
+    }
+
+    private static void assertThrows(String str)
+    {
+        try
+        {
+            new CassandraVersion(str);
+            assert false;
+        }
+        catch (IllegalArgumentException e) {}
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java b/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
deleted file mode 100644
index 4ead709..0000000
--- a/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
+++ /dev/null
@@ -1,113 +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.utils;
-
-import org.junit.Test;
-
-public class SemanticVersionTest
-{
-    @Test
-    public void testParsing()
-    {
-        SemanticVersion version;
-
-        version = new SemanticVersion("1.2.3");
-        assert version.major == 1 && version.minor == 2 && version.patch == 3;
-
-        version = new SemanticVersion("1.2.3-foo.2+Bar");
-        assert version.major == 1 && version.minor == 2 && version.patch == 3;
-    }
-
-    @Test
-    public void testComparison()
-    {
-        SemanticVersion v1, v2;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.4");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3");
-        assert v1.compareTo(v2) == 0;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("2.0.0");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3-alpha");
-        assert v1.compareTo(v2) == 1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3+foo");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3-alpha+foo");
-        assert v1.compareTo(v2) == 1;
-    }
-
-    @Test
-    public void testIsSupportedBy()
-    {
-        SemanticVersion v1, v2;
-
-        v1 = new SemanticVersion("3.0.2");
-        assert v1.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.4");
-        assert v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.3.3");
-        assert v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("2.2.3");
-        v2 = new SemanticVersion("1.3.3");
-        assert !v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("3.1.0");
-        v2 = new SemanticVersion("3.0.1");
-        assert !v1.isSupportedBy(v2);
-        assert v2.isSupportedBy(v1);
-    }
-
-    @Test
-    public void testInvalid()
-    {
-        assertThrows("1.0");
-        assertThrows("1.0.0a");
-        assertThrows("1.a.4");
-        assertThrows("1.0.0-foo&");
-    }
-
-    private static void assertThrows(String str)
-    {
-        try
-        {
-            new SemanticVersion(str);
-            assert false;
-        }
-        catch (IllegalArgumentException e) {}
-    }
-}


[2/3] cassandra git commit: Let CassandraVersion handle SNAPSHOT version

Posted by yu...@apache.org.
Let CassandraVersion handle SNAPSHOT version

patch by yukim; reviewed by Jeremiah Jordan for CASSANDRA-9438


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2385dc2d
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2385dc2d
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2385dc2d

Branch: refs/heads/trunk
Commit: 2385dc2d912391190d4f834bd43f6c1fdf368e4d
Parents: 3adfd15
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue May 26 10:50:41 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue May 26 10:50:41 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  10 +-
 .../cassandra/repair/AnticompactionTask.java    |   6 +-
 .../apache/cassandra/service/ClientState.java   |   8 +-
 .../transport/messages/StartupMessage.java      |   4 +-
 .../cassandra/utils/CassandraVersion.java       | 231 +++++++++++++++++++
 .../apache/cassandra/utils/SemanticVersion.java | 231 -------------------
 .../apache/cassandra/db/SystemKeyspaceTest.java |   4 +-
 .../cassandra/utils/CassandraVersionTest.java   | 146 ++++++++++++
 .../cassandra/utils/SemanticVersionTest.java    | 113 ---------
 11 files changed, 395 insertions(+), 361 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d4a8150..2a242c1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,7 @@
  * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
  * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)
  * Add ability to stop compaction by ID (CASSANDRA-7207)
+ * Let CassandraVersion handle SNAPSHOT version (CASSANDRA-9438)
 Merged from 2.1:
  * (cqlsh) Better float precision by default (CASSANDRA-9224)
  * Improve estimated row count (CASSANDRA-9107)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 2698a8f..7b9261c 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -56,7 +56,7 @@ import org.github.jamm.MemoryMeter;
 
 public class QueryProcessor implements QueryHandler
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.2.0");
+    public static final CassandraVersion CQL_VERSION = new CassandraVersion("3.2.0");
 
     public static final QueryProcessor instance = new QueryProcessor();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 6e754a4..67a3162 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -70,11 +70,11 @@ public final class SystemKeyspace
     // Used to indicate that there was a previous version written to the legacy (pre 1.2)
     // system.Versions table, but that we cannot read it. Suffice to say, any upgrade should
     // proceed through 1.2.x before upgrading to the current version.
-    public static final SemanticVersion UNREADABLE_VERSION = new SemanticVersion("0.0.0-unknown");
+    public static final CassandraVersion UNREADABLE_VERSION = new CassandraVersion("0.0.0-unknown");
 
     // Used to indicate that no previous version information was found. When encountered, we assume that
     // Cassandra was not previously installed and we're in the process of starting a fresh node.
-    public static final SemanticVersion NULL_VERSION = new SemanticVersion("0.0.0-absent");
+    public static final CassandraVersion NULL_VERSION = new CassandraVersion("0.0.0-absent");
 
     public static final String NAME = "system";
 
@@ -679,19 +679,19 @@ public final class SystemKeyspace
      * @param ep endpoint address to check
      * @return Release version or null if version is unknown.
      */
-    public static SemanticVersion getReleaseVersion(InetAddress ep)
+    public static CassandraVersion getReleaseVersion(InetAddress ep)
     {
         try
         {
             if (FBUtilities.getBroadcastAddress().equals(ep))
             {
-                return new SemanticVersion(FBUtilities.getReleaseVersionString());
+                return new CassandraVersion(FBUtilities.getReleaseVersionString());
             }
             String req = "SELECT release_version FROM system.%s WHERE peer=?";
             UntypedResultSet result = executeInternal(String.format(req, PEERS), ep);
             if (result != null && result.one().has("release_version"))
             {
-                return new SemanticVersion(result.one().getString("release_version"));
+                return new CassandraVersion(result.one().getString("release_version"));
             }
             // version is unknown
             return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/repair/AnticompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/AnticompactionTask.java b/src/java/org/apache/cassandra/repair/AnticompactionTask.java
index d1bbb82..16de071 100644
--- a/src/java/org/apache/cassandra/repair/AnticompactionTask.java
+++ b/src/java/org/apache/cassandra/repair/AnticompactionTask.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.AnticompactionRequest;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 public class AnticompactionTask extends AbstractFuture<InetAddress> implements Runnable
 {
@@ -39,7 +39,7 @@ public class AnticompactionTask extends AbstractFuture<InetAddress> implements R
      * Version that anticompaction response is not supported up to.
      * If Cassandra version is more than this, we need to wait for anticompaction response.
      */
-    private static final SemanticVersion VERSION_CHECKER = new SemanticVersion("2.1.5");
+    private static final CassandraVersion VERSION_CHECKER = new CassandraVersion("2.1.5");
 
     private final UUID parentSession;
     private final InetAddress neighbor;
@@ -55,7 +55,7 @@ public class AnticompactionTask extends AbstractFuture<InetAddress> implements R
     public void run()
     {
         AnticompactionRequest acr = new AnticompactionRequest(parentSession, successfulRanges);
-        SemanticVersion peerVersion = SystemKeyspace.getReleaseVersion(neighbor);
+        CassandraVersion peerVersion = SystemKeyspace.getReleaseVersion(neighbor);
         if (peerVersion != null && peerVersion.compareTo(VERSION_CHECKER) > 0)
         {
             MessagingService.instance().sendRR(acr.createMessage(), neighbor, new AnticompactionCallback(this), TimeUnit.DAYS.toMillis(1), true);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index b171f08..b9e89f5 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -43,7 +43,7 @@ import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * State related to a client connection.
@@ -51,7 +51,7 @@ import org.apache.cassandra.utils.SemanticVersion;
 public class ClientState
 {
     private static final Logger logger = LoggerFactory.getLogger(ClientState.class);
-    public static final SemanticVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
+    public static final CassandraVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
 
     private static final Set<IResource> READABLE_SYSTEM_RESOURCES = new HashSet<>();
     private static final Set<IResource> PROTECTED_AUTH_RESOURCES = new HashSet<>();
@@ -352,9 +352,9 @@ public class ClientState
         return user;
     }
 
-    public static SemanticVersion[] getCQLSupportedVersion()
+    public static CassandraVersion[] getCQLSupportedVersion()
     {
-        return new SemanticVersion[]{ QueryProcessor.CQL_VERSION };
+        return new CassandraVersion[]{ QueryProcessor.CQL_VERSION };
     }
 
     private Set<Permission> authorize(IResource resource)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index ac7a1f2..04d8e62 100644
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@ -25,7 +25,7 @@ import io.netty.buffer.ByteBuf;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.*;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * The initial message of the protocol.
@@ -70,7 +70,7 @@ public class StartupMessage extends Message.Request
 
         try 
         {
-            if (new SemanticVersion(cqlVersion).compareTo(new SemanticVersion("2.99.0")) < 0)
+            if (new CassandraVersion(cqlVersion).compareTo(new CassandraVersion("2.99.0")) < 0)
                 throw new ProtocolException(String.format("CQL version %s is not supported by the binary protocol (supported version are >= 3.0.0)", cqlVersion));
         }
         catch (IllegalArgumentException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/utils/CassandraVersion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CassandraVersion.java b/src/java/org/apache/cassandra/utils/CassandraVersion.java
new file mode 100644
index 0000000..62d68be
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/CassandraVersion.java
@@ -0,0 +1,231 @@
+/*
+ * 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.utils;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Objects;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Implements versioning used in Cassandra and CQL.
+ * <p/>
+ * Note: The following code uses a slight variation from the semver document (http://semver.org).
+ */
+public class CassandraVersion implements Comparable<CassandraVersion>
+{
+    private static final String VERSION_REGEXP = "(\\d+)\\.(\\d+)\\.(\\d+)(\\-[.\\w]+)?([.+][.\\w]+)?";
+    private static final Pattern pattern = Pattern.compile(VERSION_REGEXP);
+    private static final Pattern SNAPSHOT = Pattern.compile("-SNAPSHOT");
+
+    public final int major;
+    public final int minor;
+    public final int patch;
+
+    private final String[] preRelease;
+    private final String[] build;
+
+    private CassandraVersion(int major, int minor, int patch, String[] preRelease, String[] build)
+    {
+        this.major = major;
+        this.minor = minor;
+        this.patch = patch;
+        this.preRelease = preRelease;
+        this.build = build;
+    }
+
+    /**
+     * Parse a version from a string.
+     *
+     * @param version the string to parse
+     * @throws IllegalArgumentException if the provided string does not
+     *                                  represent a version
+     */
+    public CassandraVersion(String version)
+    {
+        String stripped = SNAPSHOT.matcher(version).replaceFirst("");
+        Matcher matcher = pattern.matcher(stripped);
+        if (!matcher.matches())
+            throw new IllegalArgumentException("Invalid version value: " + version);
+
+        try
+        {
+            this.major = Integer.parseInt(matcher.group(1));
+            this.minor = Integer.parseInt(matcher.group(2));
+            this.patch = Integer.parseInt(matcher.group(3));
+
+            String pr = matcher.group(4);
+            String bld = matcher.group(5);
+
+            this.preRelease = pr == null || pr.isEmpty() ? null : parseIdentifiers(stripped, pr);
+            this.build = bld == null || bld.isEmpty() ? null : parseIdentifiers(stripped, bld);
+        }
+        catch (NumberFormatException e)
+        {
+            throw new IllegalArgumentException("Invalid version value: " + version);
+        }
+    }
+
+    private static String[] parseIdentifiers(String version, String str)
+    {
+        // Drop initial - or +
+        str = str.substring(1);
+        String[] parts = str.split("\\.");
+        for (String part : parts)
+        {
+            if (!part.matches("\\w+"))
+                throw new IllegalArgumentException("Invalid version value: " + version);
+        }
+        return parts;
+    }
+
+    public int compareTo(CassandraVersion other)
+    {
+        if (major < other.major)
+            return -1;
+        if (major > other.major)
+            return 1;
+
+        if (minor < other.minor)
+            return -1;
+        if (minor > other.minor)
+            return 1;
+
+        if (patch < other.patch)
+            return -1;
+        if (patch > other.patch)
+            return 1;
+
+        int c = compareIdentifiers(preRelease, other.preRelease, 1);
+        if (c != 0)
+            return c;
+
+        return compareIdentifiers(build, other.build, -1);
+    }
+
+    /**
+     * Returns a version that is backward compatible with this version amongst a list
+     * of provided version, or null if none can be found.
+     * <p/>
+     * For instance:
+     * "2.0.0".findSupportingVersion("2.0.0", "3.0.0") == "2.0.0"
+     * "2.0.0".findSupportingVersion("2.1.3", "3.0.0") == "2.1.3"
+     * "2.0.0".findSupportingVersion("3.0.0") == null
+     * "2.0.3".findSupportingVersion("2.0.0") == "2.0.0"
+     * "2.1.0".findSupportingVersion("2.0.0") == null
+     */
+    public CassandraVersion findSupportingVersion(CassandraVersion... versions)
+    {
+        for (CassandraVersion version : versions)
+        {
+            if (isSupportedBy(version))
+                return version;
+        }
+        return null;
+    }
+
+    public boolean isSupportedBy(CassandraVersion version)
+    {
+        return major == version.major && this.compareTo(version) <= 0;
+    }
+
+    private static int compareIdentifiers(String[] ids1, String[] ids2, int defaultPred)
+    {
+        if (ids1 == null)
+            return ids2 == null ? 0 : defaultPred;
+        else if (ids2 == null)
+            return -defaultPred;
+
+        int min = Math.min(ids1.length, ids2.length);
+        for (int i = 0; i < min; i++)
+        {
+            Integer i1 = tryParseInt(ids1[i]);
+            Integer i2 = tryParseInt(ids2[i]);
+
+            if (i1 != null)
+            {
+                // integer have precedence
+                if (i2 == null || i1 < i2)
+                    return -1;
+                else if (i1 > i2)
+                    return 1;
+            }
+            else
+            {
+                // integer have precedence
+                if (i2 != null)
+                    return 1;
+
+                int c = ids1[i].compareTo(ids2[i]);
+                if (c != 0)
+                    return c;
+            }
+        }
+
+        if (ids1.length < ids2.length)
+            return -1;
+        if (ids1.length > ids2.length)
+            return 1;
+        return 0;
+    }
+
+    private static Integer tryParseInt(String str)
+    {
+        try
+        {
+            return Integer.valueOf(str);
+        }
+        catch (NumberFormatException e)
+        {
+            return null;
+        }
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof CassandraVersion))
+            return false;
+        CassandraVersion that = (CassandraVersion) o;
+        return major == that.major
+               && minor == that.minor
+               && patch == that.patch
+               && Arrays.equals(preRelease, that.preRelease)
+               && Arrays.equals(build, that.build);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(major, minor, patch, preRelease, build);
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(major).append('.').append(minor).append('.').append(patch);
+        if (preRelease != null)
+            sb.append('-').append(StringUtils.join(preRelease, "."));
+        if (build != null)
+            sb.append('+').append(StringUtils.join(build, "."));
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/src/java/org/apache/cassandra/utils/SemanticVersion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/SemanticVersion.java b/src/java/org/apache/cassandra/utils/SemanticVersion.java
deleted file mode 100644
index 858029d..0000000
--- a/src/java/org/apache/cassandra/utils/SemanticVersion.java
+++ /dev/null
@@ -1,231 +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.utils;
-
-import java.util.regex.Pattern;
-import java.util.regex.Matcher;
-import java.util.Arrays;
-
-import org.apache.commons.lang3.StringUtils;
-import com.google.common.base.Objects;
-
-/**
- * Implements semantic versioning as defined at http://semver.org/.
- *
- * Note: The following code uses a slight variation from the document above in
- * that it doesn't allow dashes in pre-release and build identifier.
- */
-public class SemanticVersion implements Comparable<SemanticVersion>
-{
-    private static final String VERSION_REGEXP = "(\\d+)\\.(\\d+)\\.(\\d+)(\\-[.\\w]+)?([.+][.\\w]+)?";
-    private static final Pattern pattern = Pattern.compile(VERSION_REGEXP);
-
-    public final int major;
-    public final int minor;
-    public final int patch;
-
-    private final String[] preRelease;
-    private final String[] build;
-
-    private SemanticVersion(int major, int minor, int patch, String[] preRelease, String[] build)
-    {
-        this.major = major;
-        this.minor = minor;
-        this.patch = patch;
-        this.preRelease = preRelease;
-        this.build = build;
-    }
-
-    /**
-     * Parse a semantic version from a string.
-     *
-     * @param version the string to parse
-     * @throws IllegalArgumentException if the provided string does not
-     * represent a semantic version
-     */
-    public SemanticVersion(String version)
-    {
-        Matcher matcher = pattern.matcher(version);
-        if (!matcher.matches())
-            throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-
-        try
-        {
-            this.major = Integer.parseInt(matcher.group(1));
-            this.minor = Integer.parseInt(matcher.group(2));
-            this.patch = Integer.parseInt(matcher.group(3));
-
-            String pr = matcher.group(4);
-            String bld = matcher.group(5);
-
-            this.preRelease = pr == null || pr.isEmpty() ? null : parseIdentifiers(version, pr);
-            this.build = bld == null || bld.isEmpty() ? null : parseIdentifiers(version, bld);
-
-        }
-        catch (NumberFormatException e)
-        {
-            throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-        }
-    }
-
-    private static String[] parseIdentifiers(String version, String str)
-    {
-        // Drop initial - or +
-        str = str.substring(1);
-        String[] parts = str.split("\\.");
-        for (String part : parts)
-        {
-            if (!part.matches("\\w+"))
-                throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-        }
-        return parts;
-    }
-
-    public int compareTo(SemanticVersion other)
-    {
-        if (major < other.major)
-            return -1;
-        if (major > other.major)
-            return 1;
-
-        if (minor < other.minor)
-            return -1;
-        if (minor > other.minor)
-            return 1;
-
-        if (patch < other.patch)
-            return -1;
-        if (patch > other.patch)
-            return 1;
-
-        int c = compareIdentifiers(preRelease, other.preRelease, 1);
-        if (c != 0)
-            return c;
-
-        return compareIdentifiers(build, other.build, -1);
-    }
-
-    /**
-     * Returns a version that is backward compatible with this version amongst a list
-     * of provided version, or null if none can be found.
-     *
-     * For instance:
-     *   "2.0.0".findSupportingVersion("2.0.0", "3.0.0") == "2.0.0"
-     *   "2.0.0".findSupportingVersion("2.1.3", "3.0.0") == "2.1.3"
-     *   "2.0.0".findSupportingVersion("3.0.0") == null
-     *   "2.0.3".findSupportingVersion("2.0.0") == "2.0.0"
-     *   "2.1.0".findSupportingVersion("2.0.0") == null
-     */
-    public SemanticVersion findSupportingVersion(SemanticVersion... versions)
-    {
-        for (SemanticVersion version : versions)
-        {
-            if (isSupportedBy(version))
-                return version;
-        }
-        return null;
-    }
-
-    public boolean isSupportedBy(SemanticVersion version)
-    {
-        return major == version.major && this.compareTo(version) <= 0;
-    }
-
-    private static int compareIdentifiers(String[] ids1, String[] ids2, int defaultPred)
-    {
-        if (ids1 == null)
-            return ids2 == null ? 0 : defaultPred;
-        else if (ids2 == null)
-            return -defaultPred;
-
-        int min = Math.min(ids1.length, ids2.length);
-        for (int i = 0; i < min; i++)
-        {
-            Integer i1 = tryParseInt(ids1[i]);
-            Integer i2 = tryParseInt(ids2[i]);
-
-            if (i1 != null)
-            {
-                // integer have precedence
-                if (i2 == null || i1 < i2)
-                    return -1;
-                else if (i1 > i2)
-                    return 1;
-            }
-            else
-            {
-                // integer have precedence
-                if (i2 != null)
-                    return 1;
-
-                int c = ids1[i].compareTo(ids2[i]);
-                if (c != 0)
-                    return c;
-            }
-        }
-
-        if (ids1.length < ids2.length)
-            return -1;
-        if (ids1.length > ids2.length)
-            return 1;
-        return 0;
-    }
-
-    private static Integer tryParseInt(String str)
-    {
-        try
-        {
-            return Integer.valueOf(str);
-        }
-        catch (NumberFormatException e)
-        {
-            return null;
-        }
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if(!(o instanceof SemanticVersion))
-            return false;
-        SemanticVersion that = (SemanticVersion)o;
-        return major == that.major
-            && minor == that.minor
-            && patch == that.patch
-            && Arrays.equals(preRelease, that.preRelease)
-            && Arrays.equals(build, that.build);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Objects.hashCode(major, minor, patch, preRelease, build);
-    }
-
-    @Override
-    public String toString()
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append(major).append('.').append(minor).append('.').append(patch);
-        if (preRelease != null)
-            sb.append('-').append(StringUtils.join(preRelease, "."));
-        if (build != null)
-            sb.append('+').append(StringUtils.join(build, "."));
-        return sb.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index 7531b06..093f359 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -110,7 +110,7 @@ public class SystemKeyspaceTest
     private String getOlderVersionString()
     {
         String version = FBUtilities.getReleaseVersionString();
-        SemanticVersion semver = new SemanticVersion(version.contains("-") ? version.substring(0, version.indexOf('-'))
+        CassandraVersion semver = new CassandraVersion(version.contains("-") ? version.substring(0, version.indexOf('-'))
                                                                            : version);
         return (String.format("%s.%s.%s", semver.major - 1, semver.minor, semver.patch));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java b/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
new file mode 100644
index 0000000..145b735
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.utils;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+public class CassandraVersionTest
+{
+    @Test
+    public void testParsing()
+    {
+        CassandraVersion version;
+
+        version = new CassandraVersion("1.2.3");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+
+        version = new CassandraVersion("1.2.3-foo.2+Bar");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+
+        // CassandraVersion can parse 4th '.' as build number
+        version = new CassandraVersion("1.2.3.456");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+    }
+
+    @Test
+    public void testComparison()
+    {
+        CassandraVersion v1, v2;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.4");
+        assert v1.compareTo(v2) == -1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3");
+        assert v1.compareTo(v2) == 0;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("2.0.0");
+        assert v1.compareTo(v2) == -1;
+        assert v2.compareTo(v1) == 1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3-alpha");
+        assert v1.compareTo(v2) == 1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3+foo");
+        assert v1.compareTo(v2) == -1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3-alpha+foo");
+        assert v1.compareTo(v2) == 1;
+
+        v1 = new CassandraVersion("1.2.3-alpha+1");
+        v2 = new CassandraVersion("1.2.3-alpha+2");
+        assert v1.compareTo(v2) == -1;
+    }
+
+    @Test
+    public void testIsSupportedBy()
+    {
+        CassandraVersion v1, v2;
+
+        v1 = new CassandraVersion("3.0.2");
+        assert v1.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.4");
+        assert v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.3.3");
+        assert v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("2.2.3");
+        v2 = new CassandraVersion("1.3.3");
+        assert !v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("3.1.0");
+        v2 = new CassandraVersion("3.0.1");
+        assert !v1.isSupportedBy(v2);
+        assert v2.isSupportedBy(v1);
+    }
+
+    @Test
+    public void testInvalid()
+    {
+        assertThrows("1.0");
+        assertThrows("1.0.0a");
+        assertThrows("1.a.4");
+        assertThrows("1.0.0-foo&");
+    }
+
+    @Test
+    public void testSnapshot()
+    {
+        CassandraVersion prev, next;
+
+        prev = new CassandraVersion("2.1.5");
+        next = new CassandraVersion("2.1.5.123");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = next;
+        next = new CassandraVersion("2.2.0-beta1-SNAPSHOT");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = new CassandraVersion("2.2.0-beta1");
+        next = new CassandraVersion("2.2.0-rc1-SNAPSHOT");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = next;
+        next = new CassandraVersion("2.2.0");
+        assertTrue(prev.compareTo(next) < 0);
+    }
+
+    private static void assertThrows(String str)
+    {
+        try
+        {
+            new CassandraVersion(str);
+            assert false;
+        }
+        catch (IllegalArgumentException e) {}
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2385dc2d/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java b/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
deleted file mode 100644
index 4ead709..0000000
--- a/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
+++ /dev/null
@@ -1,113 +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.utils;
-
-import org.junit.Test;
-
-public class SemanticVersionTest
-{
-    @Test
-    public void testParsing()
-    {
-        SemanticVersion version;
-
-        version = new SemanticVersion("1.2.3");
-        assert version.major == 1 && version.minor == 2 && version.patch == 3;
-
-        version = new SemanticVersion("1.2.3-foo.2+Bar");
-        assert version.major == 1 && version.minor == 2 && version.patch == 3;
-    }
-
-    @Test
-    public void testComparison()
-    {
-        SemanticVersion v1, v2;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.4");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3");
-        assert v1.compareTo(v2) == 0;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("2.0.0");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3-alpha");
-        assert v1.compareTo(v2) == 1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3+foo");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3-alpha+foo");
-        assert v1.compareTo(v2) == 1;
-    }
-
-    @Test
-    public void testIsSupportedBy()
-    {
-        SemanticVersion v1, v2;
-
-        v1 = new SemanticVersion("3.0.2");
-        assert v1.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.4");
-        assert v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.3.3");
-        assert v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("2.2.3");
-        v2 = new SemanticVersion("1.3.3");
-        assert !v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("3.1.0");
-        v2 = new SemanticVersion("3.0.1");
-        assert !v1.isSupportedBy(v2);
-        assert v2.isSupportedBy(v1);
-    }
-
-    @Test
-    public void testInvalid()
-    {
-        assertThrows("1.0");
-        assertThrows("1.0.0a");
-        assertThrows("1.a.4");
-        assertThrows("1.0.0-foo&");
-    }
-
-    private static void assertThrows(String str)
-    {
-        try
-        {
-            new SemanticVersion(str);
-            assert false;
-        }
-        catch (IllegalArgumentException e) {}
-    }
-}


[3/3] cassandra git commit: Merge branch 'cassandra-2.2' into trunk

Posted by yu...@apache.org.
Merge branch 'cassandra-2.2' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/13409fdf
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/13409fdf
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/13409fdf

Branch: refs/heads/trunk
Commit: 13409fdf07d066a79e02e487ce699932ad54884e
Parents: 2ec7594 2385dc2
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue May 26 10:51:16 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue May 26 10:51:16 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  10 +-
 .../cassandra/repair/AnticompactionTask.java    |   6 +-
 .../apache/cassandra/service/ClientState.java   |   8 +-
 .../transport/messages/StartupMessage.java      |   4 +-
 .../cassandra/utils/CassandraVersion.java       | 231 +++++++++++++++++++
 .../apache/cassandra/utils/SemanticVersion.java | 231 -------------------
 .../apache/cassandra/db/SystemKeyspaceTest.java |   4 +-
 .../cassandra/utils/CassandraVersionTest.java   | 146 ++++++++++++
 .../cassandra/utils/SemanticVersionTest.java    | 113 ---------
 11 files changed, 395 insertions(+), 361 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/13409fdf/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/13409fdf/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------