You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/04/14 17:52:49 UTC

svn commit: r764848 - in /incubator/cassandra/trunk: src/org/apache/cassandra/db/RangeFilter.java test/org/apache/cassandra/db/RangeFilterTest.java

Author: jbellis
Date: Tue Apr 14 15:52:49 2009
New Revision: 764848

URL: http://svn.apache.org/viewvc?rev=764848&view=rev
Log:
RangeFilter that filters by column name so that min <= name <= max.  patch by Jun Rao; reviewed by Todd Lipcon for #77

Added:
    incubator/cassandra/trunk/src/org/apache/cassandra/db/RangeFilter.java
    incubator/cassandra/trunk/test/org/apache/cassandra/db/RangeFilterTest.java

Added: incubator/cassandra/trunk/src/org/apache/cassandra/db/RangeFilter.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/RangeFilter.java?rev=764848&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/RangeFilter.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/RangeFilter.java Tue Apr 14 15:52:49 2009
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.cassandra.io.DataInputBuffer;
+import org.apache.cassandra.io.SSTable;
+
+/**
+ * Filters columns to satisfy colmin <= colname <= colmax
+ *
+ */
+public class RangeFilter implements IFilter
+{
+    private final String colMin_;
+    private final String colMax_;
+    private boolean isDone_;
+    int count_;
+
+    RangeFilter(String colMin, String colMax)
+    {
+        colMin_ = colMin;
+        colMax_ = colMax;
+        isDone_ = false;
+        count_ = -1;
+    }
+    
+    RangeFilter(String colMin, String colMax, int count)
+    {
+        colMin_ = colMin;
+        colMax_ = colMax;
+        isDone_ = false;
+        count_ = count;
+    }
+
+    public ColumnFamily filter(String cfName, ColumnFamily cf)
+    {
+        if (cf == null)
+            return null;
+
+        if (count_ == 0)
+        {
+            isDone_ = true;
+            return null;
+        }
+
+        ColumnFamily filteredColumnFamily = new ColumnFamily(cfName);
+
+        Collection<IColumn> columns = cf.getAllColumns();
+        for (IColumn c : columns)
+        {
+            if (c.name().compareTo(colMin_) >= 0
+                    && c.name().compareTo(colMax_) <= 0)
+            {
+                filteredColumnFamily.addColumn(c);
+                if (count_ > 0)
+                    count_--;
+                if (count_==0)
+                {
+                    isDone_ = true;
+                    break;
+                }
+            }
+        }
+        return filteredColumnFamily;
+    }
+
+    public IColumn filter(IColumn column, DataInputStream dis)
+            throws IOException
+    {
+        if (column == null || isDone_)
+            return null;
+
+        if (column.name().compareTo(colMin_) >= 0
+                && column.name().compareTo(colMax_) <= 0)
+        {
+            if (count_ > 0)
+                count_--;
+            if (count_ == 0)
+                isDone_ = true;
+            return column;
+        } else
+        {
+            return null;
+        }
+    }
+
+    public boolean isDone()
+    {
+        return isDone_;
+    }
+
+    public DataInputBuffer next(String key, String cf, SSTable ssTable)
+            throws IOException
+    {
+        return ssTable.next(key, cf);
+    }
+
+}

Added: incubator/cassandra/trunk/test/org/apache/cassandra/db/RangeFilterTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/org/apache/cassandra/db/RangeFilterTest.java?rev=764848&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/org/apache/cassandra/db/RangeFilterTest.java (added)
+++ incubator/cassandra/trunk/test/org/apache/cassandra/db/RangeFilterTest.java Tue Apr 14 15:52:49 2009
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class RangeFilterTest
+{
+    @Test
+    public void testRangeFilterOnColumns() throws IOException
+    {
+        ColumnFamily cf = new ColumnFamily("Standard1", "Standard");
+        byte[] val = "test value".getBytes();
+        cf.addColumn(new Column("a", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("b", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("c", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("d", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("e", val, System.currentTimeMillis()));
+
+        IFilter f = new RangeFilter("b", "d");
+        ColumnFamily filteredCf = f.filter(cf.name(), cf);
+        
+        Assert.assertEquals(filteredCf.getColumnCount(),3);
+        Assert.assertFalse(f.isDone());
+    }
+    
+    @Test
+    public void testRangeFilterOnColumnsWithCount() throws IOException
+    {
+        ColumnFamily cf = new ColumnFamily("Standard1", "Standard");
+        byte[] val = "test value".getBytes();
+        cf.addColumn(new Column("a", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("b", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("c", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("d", val, System.currentTimeMillis()));
+        cf.addColumn(new Column("e", val, System.currentTimeMillis()));
+
+        IFilter f = new RangeFilter("b", "d", 2);
+        ColumnFamily filteredCf = f.filter(cf.name(), cf);
+        
+        Assert.assertEquals(filteredCf.getColumnCount(),2);
+        Assert.assertTrue(f.isDone());
+    }
+
+    @Test
+    public void testRangeFilterOnSuperColumns() throws IOException
+    {
+        ColumnFamily cf = new ColumnFamily("Super1", "Super");
+        byte[] val = "test value".getBytes();
+        SuperColumn sc = null;
+        sc = new SuperColumn("a");
+        sc.addColumn("a1", new Column("a1", val, System.currentTimeMillis()));
+        sc.addColumn("a2", new Column("a2", val, System.currentTimeMillis()));
+        cf.addColumn(sc);
+        sc = new SuperColumn("b");
+        sc.addColumn("b1", new Column("b1", val, System.currentTimeMillis()));
+        sc.addColumn("b2", new Column("b2", val, System.currentTimeMillis()));
+        cf.addColumn(sc);
+        sc = new SuperColumn("c");
+        sc.addColumn("c1", new Column("c1", val, System.currentTimeMillis()));
+        sc.addColumn("c2", new Column("c2", val, System.currentTimeMillis()));
+        cf.addColumn(sc);
+        sc = new SuperColumn("d");
+        sc.addColumn("d1", new Column("d1", val, System.currentTimeMillis()));
+        sc.addColumn("d2", new Column("d2", val, System.currentTimeMillis()));
+        cf.addColumn(sc);
+        sc = new SuperColumn("e");
+        sc.addColumn("e1", new Column("e1", val, System.currentTimeMillis()));
+        sc.addColumn("e2", new Column("e2", val, System.currentTimeMillis()));
+        cf.addColumn(sc);
+
+        IFilter f = new RangeFilter("b", "d");
+        ColumnFamily filteredCf = f.filter(cf.name(), cf);
+
+        IColumn col = filteredCf.getColumn("a");
+        Assert.assertNull(col);
+
+        col = filteredCf.getColumn("e");
+        Assert.assertNull(col);
+
+        col = filteredCf.getColumn("c");
+        Assert.assertNotNull(col);
+        Assert.assertFalse(f.isDone());
+    }
+
+}