Skip to content

Commit

Permalink
Rename NullableBigintState to NullableLongState
Browse files Browse the repository at this point in the history
  • Loading branch information
haozhun committed Mar 10, 2015
1 parent 9ecb1cb commit 2915037
Show file tree
Hide file tree
Showing 8 changed files with 22 additions and 22 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
package com.facebook.presto.operator.aggregation;

import com.facebook.presto.operator.aggregation.state.InitialLongValue;
import com.facebook.presto.operator.aggregation.state.NullableBigintState;
import com.facebook.presto.operator.aggregation.state.NullableLongState;
import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.type.SqlType;

Expand All @@ -32,7 +32,7 @@ public static void max(BigintMaxState state, @SqlType(StandardTypes.BIGINT) long
}

public interface BigintMaxState
extends NullableBigintState
extends NullableLongState
{
@Override
@InitialLongValue(Long.MIN_VALUE)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
package com.facebook.presto.operator.aggregation;

import com.facebook.presto.operator.aggregation.state.InitialLongValue;
import com.facebook.presto.operator.aggregation.state.NullableBigintState;
import com.facebook.presto.operator.aggregation.state.NullableLongState;
import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.type.SqlType;

Expand All @@ -32,7 +32,7 @@ public static void min(BigintMinState state, @SqlType(StandardTypes.BIGINT) long
}

public interface BigintMinState
extends NullableBigintState
extends NullableLongState
{
@Override
@InitialLongValue(Long.MAX_VALUE)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
*/
package com.facebook.presto.operator.aggregation;

import com.facebook.presto.operator.aggregation.state.NullableBigintState;
import com.facebook.presto.operator.aggregation.state.NullableLongState;
import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.type.SqlType;

Expand All @@ -26,7 +26,7 @@ private LongSumAggregation() {}

@InputFunction
@IntermediateInputFunction
public static void sum(NullableBigintState state, @SqlType(StandardTypes.BIGINT) long value)
public static void sum(NullableLongState state, @SqlType(StandardTypes.BIGINT) long value)
{
state.setNull(false);
state.setLong(state.getLong() + value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,8 @@
*/
package com.facebook.presto.operator.aggregation.state;

@AccumulatorStateMetadata(stateSerializerClass = NullableBigintStateSerializer.class)
public interface NullableBigintState
@AccumulatorStateMetadata(stateSerializerClass = NullableLongStateSerializer.class)
public interface NullableLongState
extends AccumulatorState
{
long getLong();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@

import static com.facebook.presto.spi.type.BigintType.BIGINT;

public class NullableBigintStateSerializer
implements AccumulatorStateSerializer<NullableBigintState>
public class NullableLongStateSerializer
implements AccumulatorStateSerializer<NullableLongState>
{
@Override
public Type getSerializedType()
Expand All @@ -29,7 +29,7 @@ public Type getSerializedType()
}

@Override
public void serialize(NullableBigintState state, BlockBuilder out)
public void serialize(NullableLongState state, BlockBuilder out)
{
if (state.isNull()) {
out.appendNull();
Expand All @@ -40,7 +40,7 @@ public void serialize(NullableBigintState state, BlockBuilder out)
}

@Override
public void deserialize(Block block, int index, NullableBigintState state)
public void deserialize(Block block, int index, NullableLongState state)
{
state.setNull(block.isNull(index));
if (!state.isNull()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
package com.facebook.presto.operator.aggregation;

import com.facebook.presto.metadata.FunctionListBuilder;
import com.facebook.presto.operator.aggregation.state.NullableBigintState;
import com.facebook.presto.operator.aggregation.state.NullableLongState;
import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.block.BlockBuilder;
import com.facebook.presto.spi.block.BlockBuilderStatus;
Expand Down Expand Up @@ -69,7 +69,7 @@ public static final class CountNull
private CountNull() {}

@InputFunction
public static void input(NullableBigintState state, @NullablePosition @SqlType(StandardTypes.BIGINT) Block block, @BlockIndex int position)
public static void input(NullableLongState state, @NullablePosition @SqlType(StandardTypes.BIGINT) Block block, @BlockIndex int position)
{
if (block.isNull(position)) {
state.setLong(state.getLong() + 1);
Expand All @@ -78,7 +78,7 @@ public static void input(NullableBigintState state, @NullablePosition @SqlType(S
}

@CombineFunction
public static void combine(NullableBigintState state, NullableBigintState scratchState)
public static void combine(NullableLongState state, NullableLongState scratchState)
{
state.setLong(state.getLong() + scratchState.getLong());
state.setNull(state.isNull() && scratchState.isNull());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
import com.facebook.presto.operator.aggregation.state.AccumulatorStateFactory;
import com.facebook.presto.operator.aggregation.state.AccumulatorStateSerializer;
import com.facebook.presto.operator.aggregation.state.LongState;
import com.facebook.presto.operator.aggregation.state.NullableBigintState;
import com.facebook.presto.operator.aggregation.state.NullableLongState;
import com.facebook.presto.operator.aggregation.state.StateCompiler;
import com.facebook.presto.operator.aggregation.state.VarianceState;
import com.facebook.presto.spi.block.Block;
Expand All @@ -38,10 +38,10 @@ public void testPrimitiveNullableLongSerialization()
{
StateCompiler compiler = new StateCompiler();

AccumulatorStateFactory<NullableBigintState> factory = compiler.generateStateFactory(NullableBigintState.class);
AccumulatorStateSerializer<NullableBigintState> serializer = compiler.generateStateSerializer(NullableBigintState.class);
NullableBigintState state = factory.createSingleState();
NullableBigintState deserializedState = factory.createSingleState();
AccumulatorStateFactory<NullableLongState> factory = compiler.generateStateFactory(NullableLongState.class);
AccumulatorStateSerializer<NullableLongState> serializer = compiler.generateStateSerializer(NullableLongState.class);
NullableLongState state = factory.createSingleState();
NullableLongState deserializedState = factory.createSingleState();

state.setLong(2);
state.setNull(false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import com.facebook.presto.operator.aggregation.AggregationFunction;
import com.facebook.presto.operator.aggregation.InputFunction;
import com.facebook.presto.operator.aggregation.IntermediateInputFunction;
import com.facebook.presto.operator.aggregation.state.NullableBigintState;
import com.facebook.presto.operator.aggregation.state.NullableLongState;
import com.facebook.presto.spi.type.StandardTypes;
import com.facebook.presto.type.SqlType;

Expand All @@ -27,7 +27,7 @@ private CustomSum() {}

@InputFunction
@IntermediateInputFunction
public static void input(NullableBigintState state, @SqlType(StandardTypes.BIGINT) long value)
public static void input(NullableLongState state, @SqlType(StandardTypes.BIGINT) long value)
{
state.setLong(state.getLong() + value);
state.setNull(false);
Expand Down

0 comments on commit 2915037

Please sign in to comment.