/**
* 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 org.apache.commons.lang.NotImplementedException;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.utils.ByteBufferUtil;
/** for sorting columns representing row keys in the row ordering as determined by a partitioner.
* Not intended for user-defined CFs, and will in fact error out if used with such. */
public class LocalByPartionerType<T extends Token> extends AbstractType<ByteBuffer>
{
private final IPartitioner<T> partitioner;
public LocalByPartionerType(IPartitioner<T> partitioner)
{
this.partitioner = partitioner;
}
public ByteBuffer compose(ByteBuffer bytes)
{
throw new UnsupportedOperationException("You can't do this with a local partitioner.");
}
public ByteBuffer decompose(ByteBuffer bytes)
{
throw new UnsupportedOperationException("You can't do this with a local partitioner.");
}
public String getString(ByteBuffer bytes)
{
return ByteBufferUtil.bytesToHex(bytes);
}
public String toString(ByteBuffer bb)
{
return getString(bb);
}
public ByteBuffer fromString(String source)
{
throw new NotImplementedException();
}
public int compare(ByteBuffer o1, ByteBuffer o2)
{
return partitioner.decorateKey(o1).compareTo(partitioner.decorateKey(o2));
}
public void validate(ByteBuffer bytes) throws MarshalException
{
throw new IllegalStateException("You shouldn't be validating this.");
}
public Class<ByteBuffer> getType()
{
return ByteBuffer.class;
}
public boolean isSigned()
{
throw new UnsupportedOperationException();
}
public boolean isCaseSensitive()
{
throw new UnsupportedOperationException();
}
public boolean isCurrency()
{
throw new UnsupportedOperationException();
}
public int getPrecision(ByteBuffer obj)
{
throw new UnsupportedOperationException();
}
public int getScale(ByteBuffer obj)
{
throw new UnsupportedOperationException();
}
public int getJdbcType()
{
throw new UnsupportedOperationException();
}
public boolean needsQuotes()
{
throw new UnsupportedOperationException();
}
}