package org.apache.cassandra.db.filter; /* * * 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. * */ import java.nio.ByteBuffer; import java.util.Comparator; import java.util.Iterator; import java.util.SortedSet; import org.apache.cassandra.db.*; import org.apache.cassandra.db.columniterator.IColumnIterator; import org.apache.cassandra.db.columniterator.SSTableNamesIterator; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.io.sstable.SSTableReader; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.utils.FBUtilities; public class NamesQueryFilter implements IFilter { public final SortedSet<ByteBuffer> columns; public NamesQueryFilter(SortedSet<ByteBuffer> columns) { this.columns = columns; } public NamesQueryFilter(ByteBuffer column) { this(FBUtilities.singleton(column)); } public IColumnIterator getMemtableColumnIterator(ColumnFamily cf, DecoratedKey key, AbstractType comparator) { return Memtable.getNamesIterator(key, cf, this); } public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey key) { return new SSTableNamesIterator(sstable, key, columns); } public IColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key) { return new SSTableNamesIterator(sstable, file, key, columns); } public SuperColumn filterSuperColumn(SuperColumn superColumn, int gcBefore) { for (IColumn column : superColumn.getSubColumns()) { if (!columns.contains(column.name()) || !QueryFilter.isRelevant(column, superColumn, gcBefore)) { superColumn.remove(column.name()); } } return superColumn; } public void collectReducedColumns(IColumnContainer container, Iterator<IColumn> reducedColumns, int gcBefore) { while (reducedColumns.hasNext()) { IColumn column = reducedColumns.next(); if (QueryFilter.isRelevant(column, container, gcBefore)) container.addColumn(column); } } public Comparator<IColumn> getColumnComparator(AbstractType comparator) { return comparator.columnComparator; } }