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 2012/07/27 17:19:39 UTC

[45/50] [abbrv] Add lists, sets and maps support

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91bdf7fb/src/java/org/apache/cassandra/db/marshal/SetType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
new file mode 100644
index 0000000..7a72fe9
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -0,0 +1,134 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cql3.ColumnNameBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.IColumn;
+import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+
+public class SetType extends CollectionType
+{
+    private static final Logger logger = LoggerFactory.getLogger(SetType.class);
+
+    // interning instances
+    private static final Map<AbstractType<?>, SetType> instances = new HashMap<AbstractType<?>, SetType>();
+
+    public final AbstractType<?> elements;
+
+    public static SetType getInstance(TypeParser parser) throws ConfigurationException
+    {
+        List<AbstractType<?>> l = parser.getTypeParameters();
+        if (l.size() != 1)
+            throw new ConfigurationException("SetType takes exactly 1 type parameter");
+
+        return getInstance(l.get(0));
+    }
+
+    public static synchronized SetType getInstance(AbstractType<?> elements)
+    {
+        SetType t = instances.get(elements);
+        if (t == null)
+        {
+            t = new SetType(elements);
+            instances.put(elements, t);
+        }
+        return t;
+    }
+
+    public SetType(AbstractType<?> elements)
+    {
+        super(Kind.SET);
+        this.elements = elements;
+    }
+
+    protected AbstractType<?> nameComparator()
+    {
+        return elements;
+    }
+
+    protected AbstractType<?> valueComparator()
+    {
+        return EmptyType.instance;
+    }
+
+    protected void appendToStringBuilder(StringBuilder sb)
+    {
+        sb.append(getClass().getName()).append(TypeParser.stringifyTypeParameters(Collections.<AbstractType<?>>singletonList(elements)));
+    }
+
+    public void executeFunction(ColumnFamily cf, ColumnNameBuilder fullPath, Function fct, List<Term> args, UpdateParameters params) throws InvalidRequestException
+    {
+        switch (fct)
+        {
+            case ADD:
+                doAdd(cf, fullPath, args, params);
+                break;
+            case DISCARD_SET:
+                doDiscard(cf, fullPath, args, params);
+                break;
+            default:
+                throw new AssertionError("Unsupported function " + fct);
+        }
+    }
+
+    public void doAdd(ColumnFamily cf, ColumnNameBuilder builder, List<Term> values, UpdateParameters params) throws InvalidRequestException
+    {
+        for (int i = 0; i < values.size(); ++i)
+        {
+            ColumnNameBuilder b = i == values.size() - 1 ? builder : builder.copy();
+            ByteBuffer name = b.add(values.get(i).getByteBuffer(elements, params.variables)).build();
+            cf.addColumn(params.makeColumn(name, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        }
+    }
+
+    public void doDiscard(ColumnFamily cf, ColumnNameBuilder builder, List<Term> values, UpdateParameters params) throws InvalidRequestException
+    {
+        for (int i = 0; i < values.size(); ++i)
+        {
+            ColumnNameBuilder b = i == values.size() - 1 ? builder : builder.copy();
+            ByteBuffer name = b.add(values.get(i).getByteBuffer(elements, params.variables)).build();
+            cf.addColumn(params.makeTombstone(name));
+        }
+    }
+
+    public ByteBuffer serializeForThrift(List<Pair<ByteBuffer, IColumn>> columns)
+    {
+        // We're using a list for now, since json doesn't have maps
+        List<Object> l = new ArrayList<Object>(columns.size());
+        for (Pair<ByteBuffer, IColumn> p : columns)
+            l.add(elements.compose(p.left));
+        return ByteBufferUtil.bytes(FBUtilities.json(l));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91bdf7fb/src/java/org/apache/cassandra/db/marshal/TypeParser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TypeParser.java b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
index 649e739..7bbd6a7 100644
--- a/src/java/org/apache/cassandra/db/marshal/TypeParser.java
+++ b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.marshal;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -29,6 +30,8 @@ import java.util.Map;
 import org.apache.commons.lang.StringUtils;
 
 import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.cql3.CFPropDefs;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -50,6 +53,11 @@ public class TypeParser
         this.idx = idx;
     }
 
+    public TypeParser(String str)
+    {
+        this(str, 0);
+    }
+
     /**
      * Parse a string containing an type definition.
      */
@@ -99,7 +107,7 @@ public class TypeParser
     /**
      * Parse an AbstractType from current position of this parser.
      */
-    private AbstractType<?> parse() throws ConfigurationException
+    public AbstractType<?> parse() throws ConfigurationException
     {
         skipBlank();
         String name = readNextIdentifier();
@@ -231,6 +239,60 @@ public class TypeParser
         throw new ConfigurationException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
     }
 
+    public Map<ByteBuffer, CollectionType> getCollectionsParameters() throws ConfigurationException
+    {
+        Map<ByteBuffer, CollectionType> map = new HashMap<ByteBuffer, CollectionType>();
+
+        if (isEOS())
+            return map;
+
+        if (str.charAt(idx) != '(')
+            throw new IllegalStateException();
+
+        ++idx; // skipping '('
+
+        while (skipBlankAndComma())
+        {
+            if (str.charAt(idx) == ')')
+            {
+                ++idx;
+                return map;
+            }
+
+            String bbHex = readNextIdentifier();
+            ByteBuffer bb = null;
+            try
+            {
+                 bb = ByteBufferUtil.hexToBytes(bbHex);
+            }
+            catch (NumberFormatException e)
+            {
+                throwSyntaxError(e.getMessage());
+            }
+
+            skipBlank();
+            if (str.charAt(idx) != ':')
+                throwSyntaxError("expecting ':' token");
+
+            ++idx;
+            skipBlank();
+            try
+            {
+                AbstractType<?> type = parse();
+                if (!(type instanceof CollectionType))
+                    throw new ConfigurationException(type.toString() + " is not a collection type");
+                map.put(bb, (CollectionType)type);
+            }
+            catch (ConfigurationException e)
+            {
+                ConfigurationException ex = new ConfigurationException(String.format("Exception while parsing '%s' around char %d", str, idx));
+                ex.initCause(e);
+                throw ex;
+            }
+        }
+        throw new ConfigurationException(String.format("Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
+    }
+
     private static AbstractType<?> getAbstractType(String compareWith) throws ConfigurationException
     {
         String className = compareWith.contains(".") ? compareWith : "org.apache.cassandra.db.marshal." + compareWith;
@@ -388,7 +450,7 @@ public class TypeParser
     }
 
     // left idx positioned on the character stopping the read
-    private String readNextIdentifier()
+    public String readNextIdentifier()
     {
         int i = idx;
         while (!isEOS() && isIdentifierChar(str.charAt(idx)))
@@ -397,6 +459,12 @@ public class TypeParser
         return str.substring(i, idx);
     }
 
+    public char readNextChar()
+    {
+        skipBlank();
+        return str.charAt(idx++);
+    }
+
     /**
      * Helper function to ease the writing of AbstractType.toString() methods.
      */
@@ -428,4 +496,23 @@ public class TypeParser
         sb.append('(').append(StringUtils.join(types, ",")).append(')');
         return sb.toString();
     }
+
+    public static String stringifyCollectionsParameters(Map<ByteBuffer, CollectionType> collections)
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append('(');
+        boolean first = true;
+        for (Map.Entry<ByteBuffer, CollectionType> entry : collections.entrySet())
+        {
+            if (first)
+            {
+                sb.append(',');
+                first = false;
+            }
+            sb.append(ByteBufferUtil.bytesToHex(entry.getKey())).append(":");
+            entry.getValue().appendToStringBuilder(sb);
+        }
+        sb.append(')');
+        return sb.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/91bdf7fb/src/java/org/apache/cassandra/utils/UUIDGen.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index 02d12d9..2f473d7 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -154,6 +154,25 @@ public class UUIDGen
         return createTimeUUIDBytes(instance.createTimeUnsafe(timeMillis));
     }
 
+    /**
+     * Converts a 100-nanoseconds precision timestamp into the 16 byte representation
+     * of a type 1 UUID (a time-based UUID).
+     *
+     * To specify a 100-nanoseconds precision timestamp, one should provide a milliseconds timestamp and
+     * a number 0 <= n < 10000 such that n*100 is the number of nanoseconds within that millisecond.
+     *
+     * <p><i><b>Warning:</b> This method is not guaranteed to return unique UUIDs; Multiple
+     * invocations using identical timestamps will result in identical UUIDs.</i></p>
+     *
+     * @return a type 1 UUID represented as a byte[]
+     */
+    public static byte[] getTimeUUIDBytes(long timeMillis, int nanos)
+    {
+        if (nanos >= 10000)
+            throw new IllegalArgumentException();
+        return createTimeUUIDBytes(instance.createTimeUnsafe(timeMillis, nanos));
+    }
+
     private static byte[] createTimeUUIDBytes(long msb)
     {
         long lsb = instance.getClockSeqAndNode(FBUtilities.getLocalAddress());
@@ -208,7 +227,12 @@ public class UUIDGen
 
     private long createTimeUnsafe(long when)
     {
-        long nanosSince = (when - START_EPOCH) * 10000;
+        return createTimeUnsafe(when, 0);
+    }
+
+    private long createTimeUnsafe(long when, int nanos)
+    {
+        long nanosSince = ((when - START_EPOCH) * 10000) + nanos;
         return createTime(nanosSince);
     }