org.apache.cassandra.cql3.functions.CastFcts Maven / Gradle / Ivy
Go to download
Show more of this group Show more artifacts with this name
Show all versions of cassandra-all Show documentation
Show all versions of cassandra-all Show documentation
The Apache Cassandra Project develops a highly scalable second-generation distributed database, bringing together Dynamo's fully distributed design and Bigtable's ColumnFamily-based data model.
/*
* 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.cql3.functions;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.cassandra.cql3.CQL3Type;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.AsciiType;
import org.apache.cassandra.db.marshal.BooleanType;
import org.apache.cassandra.db.marshal.ByteType;
import org.apache.cassandra.db.marshal.CounterColumnType;
import org.apache.cassandra.db.marshal.DecimalType;
import org.apache.cassandra.db.marshal.DoubleType;
import org.apache.cassandra.db.marshal.FloatType;
import org.apache.cassandra.db.marshal.InetAddressType;
import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.db.marshal.IntegerType;
import org.apache.cassandra.db.marshal.LongType;
import org.apache.cassandra.db.marshal.ShortType;
import org.apache.cassandra.db.marshal.SimpleDateType;
import org.apache.cassandra.db.marshal.TimeType;
import org.apache.cassandra.db.marshal.TimeUUIDType;
import org.apache.cassandra.db.marshal.TimestampType;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.db.marshal.UUIDType;
import org.apache.cassandra.transport.ProtocolVersion;
import static org.apache.cassandra.cql3.functions.TimeFcts.*;
import org.apache.commons.lang3.text.WordUtils;
/**
* Casting functions
*
*/
public final class CastFcts
{
private static final String FUNCTION_NAME_PREFIX = "castAs";
public static Collection all()
{
List functions = new ArrayList<>();
@SuppressWarnings("unchecked")
final AbstractType extends Number>[] numericTypes = new AbstractType[] {ByteType.instance,
ShortType.instance,
Int32Type.instance,
LongType.instance,
FloatType.instance,
DoubleType.instance,
DecimalType.instance,
CounterColumnType.instance,
IntegerType.instance};
for (AbstractType extends Number> inputType : numericTypes)
{
addFunctionIfNeeded(functions, inputType, ByteType.instance, Number::byteValue);
addFunctionIfNeeded(functions, inputType, ShortType.instance, Number::shortValue);
addFunctionIfNeeded(functions, inputType, Int32Type.instance, Number::intValue);
addFunctionIfNeeded(functions, inputType, LongType.instance, Number::longValue);
addFunctionIfNeeded(functions, inputType, FloatType.instance, Number::floatValue);
addFunctionIfNeeded(functions, inputType, DoubleType.instance, Number::doubleValue);
addFunctionIfNeeded(functions, inputType, DecimalType.instance, getDecimalConversionFunction(inputType));
addFunctionIfNeeded(functions, inputType, IntegerType.instance, p -> BigInteger.valueOf(p.longValue()));
functions.add(CastAsTextFunction.create(inputType, AsciiType.instance));
functions.add(CastAsTextFunction.create(inputType, UTF8Type.instance));
}
functions.add(JavaFunctionWrapper.create(AsciiType.instance, UTF8Type.instance, p -> p));
functions.add(CastAsTextFunction.create(InetAddressType.instance, AsciiType.instance));
functions.add(CastAsTextFunction.create(InetAddressType.instance, UTF8Type.instance));
functions.add(CastAsTextFunction.create(BooleanType.instance, AsciiType.instance));
functions.add(CastAsTextFunction.create(BooleanType.instance, UTF8Type.instance));
functions.add(CassandraFunctionWrapper.create(TimeUUIDType.instance, SimpleDateType.instance, toDate(TimeUUIDType.instance)));
functions.add(CassandraFunctionWrapper.create(TimeUUIDType.instance, TimestampType.instance, toTimestamp(TimeUUIDType.instance)));
functions.add(CastAsTextFunction.create(TimeUUIDType.instance, AsciiType.instance));
functions.add(CastAsTextFunction.create(TimeUUIDType.instance, UTF8Type.instance));
functions.add(CassandraFunctionWrapper.create(TimestampType.instance, SimpleDateType.instance, toDate(TimestampType.instance)));
functions.add(CastAsTextFunction.create(TimestampType.instance, AsciiType.instance));
functions.add(CastAsTextFunction.create(TimestampType.instance, UTF8Type.instance));
functions.add(CassandraFunctionWrapper.create(SimpleDateType.instance, TimestampType.instance, toTimestamp(SimpleDateType.instance)));
functions.add(CastAsTextFunction.create(SimpleDateType.instance, AsciiType.instance));
functions.add(CastAsTextFunction.create(SimpleDateType.instance, UTF8Type.instance));
functions.add(CastAsTextFunction.create(TimeType.instance, AsciiType.instance));
functions.add(CastAsTextFunction.create(TimeType.instance, UTF8Type.instance));
functions.add(CastAsTextFunction.create(UUIDType.instance, AsciiType.instance));
functions.add(CastAsTextFunction.create(UUIDType.instance, UTF8Type.instance));
return functions;
}
/**
* Returns the conversion function to convert the specified type into a Decimal type
*
* @param inputType the input type
* @return the conversion function to convert the specified type into a Decimal type
*/
private static java.util.function.Function getDecimalConversionFunction(AbstractType extends Number> inputType)
{
if (inputType == FloatType.instance)
return p -> new BigDecimal(Float.toString(p.floatValue()));
if (inputType == DoubleType.instance)
return p -> BigDecimal.valueOf(p.doubleValue());
if (inputType == IntegerType.instance)
return p -> new BigDecimal((BigInteger) p);
return p -> BigDecimal.valueOf(p.longValue());
}
/**
* Creates the name of the cast function use to cast to the specified type.
*
* @param outputType the output type
* @return the name of the cast function use to cast to the specified type
*/
public static String getFunctionName(AbstractType> outputType)
{
return getFunctionName(outputType.asCQL3Type());
}
/**
* Creates the name of the cast function use to cast to the specified type.
*
* @param outputType the output type
* @return the name of the cast function use to cast to the specified type
*/
public static String getFunctionName(CQL3Type outputType)
{
return FUNCTION_NAME_PREFIX + WordUtils.capitalize(toLowerCaseString(outputType));
}
/**
* Adds to the list a function converting the input type in to the output type if they are not the same.
*
* @param functions the list to add to
* @param inputType the input type
* @param outputType the output type
* @param converter the function use to convert the input type into the output type
*/
private static void addFunctionIfNeeded(List functions,
AbstractType inputType,
AbstractType outputType,
java.util.function.Function converter)
{
if (!inputType.equals(outputType))
functions.add(wrapJavaFunction(inputType, outputType, converter));
}
@SuppressWarnings("unchecked")
private static Function wrapJavaFunction(AbstractType inputType,
AbstractType outputType,
java.util.function.Function converter)
{
return inputType.equals(CounterColumnType.instance)
? JavaCounterFunctionWrapper.create(outputType, (java.util.function.Function) converter)
: JavaFunctionWrapper.create(inputType, outputType, converter);
}
private static String toLowerCaseString(CQL3Type type)
{
return type.toString().toLowerCase();
}
/**
* Base class for the CAST functions.
*
* @param the input type
* @param the output type
*/
private static abstract class CastFunction extends NativeScalarFunction
{
public CastFunction(AbstractType inputType, AbstractType outputType)
{
super(getFunctionName(outputType), outputType, inputType);
}
@Override
public String columnName(List columnNames)
{
return String.format("cast(%s as %s)", columnNames.get(0), toLowerCaseString(outputType().asCQL3Type()));
}
@SuppressWarnings("unchecked")
protected AbstractType outputType()
{
return (AbstractType) returnType;
}
@SuppressWarnings("unchecked")
protected AbstractType inputType()
{
return (AbstractType) argTypes.get(0);
}
}
/**
* CastFunction
that implements casting by wrapping a java Function
.
*
* @param the input parameter
* @param the output parameter
*/
private static class JavaFunctionWrapper extends CastFunction
{
/**
* The java function used to convert the input type into the output one.
*/
private final java.util.function.Function converter;
public static JavaFunctionWrapper create(AbstractType inputType,
AbstractType outputType,
java.util.function.Function converter)
{
return new JavaFunctionWrapper(inputType, outputType, converter);
}
protected JavaFunctionWrapper(AbstractType inputType,
AbstractType outputType,
java.util.function.Function converter)
{
super(inputType, outputType);
this.converter = converter;
}
public final ByteBuffer execute(ProtocolVersion protocolVersion, List parameters)
{
ByteBuffer bb = parameters.get(0);
if (bb == null)
return null;
return outputType().decompose(converter.apply(compose(bb)));
}
protected I compose(ByteBuffer bb)
{
return inputType().compose(bb);
}
}
/**
* JavaFunctionWrapper
for counter columns.
*
* Counter columns need to be handled in a special way because their binary representation is converted into
* the one of a BIGINT before functions are applied.
*
* @param the output parameter
*/
private static class JavaCounterFunctionWrapper extends JavaFunctionWrapper
{
public static JavaFunctionWrapper create(AbstractType outputType,
java.util.function.Function converter)
{
return new JavaCounterFunctionWrapper(outputType, converter);
}
protected JavaCounterFunctionWrapper(AbstractType outputType,
java.util.function.Function converter)
{
super(CounterColumnType.instance, outputType, converter);
}
protected Long compose(ByteBuffer bb)
{
return LongType.instance.compose(bb);
}
}
/**
* CastFunction
that implements casting by wrapping an existing NativeScalarFunction
.
*
* @param the input parameter
* @param the output parameter
*/
private static final class CassandraFunctionWrapper extends CastFunction
{
/**
* The native scalar function used to perform the conversion.
*/
private final NativeScalarFunction delegate;
public static CassandraFunctionWrapper create(AbstractType inputType,
AbstractType outputType,
NativeScalarFunction delegate)
{
return new CassandraFunctionWrapper(inputType, outputType, delegate);
}
private CassandraFunctionWrapper(AbstractType inputType,
AbstractType outputType,
NativeScalarFunction delegate)
{
super(inputType, outputType);
assert delegate.argTypes().size() == 1 && inputType.equals(delegate.argTypes().get(0));
assert outputType.equals(delegate.returnType());
this.delegate = delegate;
}
public ByteBuffer execute(ProtocolVersion protocolVersion, List parameters)
{
return delegate.execute(protocolVersion, parameters);
}
}
/**
* CastFunction
that can be used to cast a type into ascii or text types.
*
* @param the input parameter
*/
private static final class CastAsTextFunction extends CastFunction
{
public static CastAsTextFunction create(AbstractType inputType,
AbstractType outputType)
{
return new CastAsTextFunction(inputType, outputType);
}
private CastAsTextFunction(AbstractType inputType,
AbstractType outputType)
{
super(inputType, outputType);
}
public ByteBuffer execute(ProtocolVersion protocolVersion, List parameters)
{
ByteBuffer bb = parameters.get(0);
if (bb == null)
return null;
return outputType().decompose(inputType().getSerializer().toCQLLiteral(bb));
}
}
/**
* The class must not be instantiated as it contains only static variables.
*/
private CastFcts()
{
}
}