additional UTs for [DoubleMax/DoubleMin] aggregation

This commit is contained in:
Himanshu Gupta 2015-05-12 23:37:55 -05:00
parent d0ec945129
commit cebb550796
4 changed files with 266 additions and 98 deletions

View File

@ -0,0 +1,133 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.query.aggregation;
import com.google.common.primitives.Doubles;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.ColumnSelectorFactory;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.nio.ByteBuffer;
/**
*/
public class DoubleMaxAggregationTest
{
private DoubleMaxAggregatorFactory doubleMaxAggFactory;
private ColumnSelectorFactory colSelectorFactory;
private TestFloatColumnSelector selector;
private float[] values = {1.1f, 2.7f, 3.5f, 1.3f};
public DoubleMaxAggregationTest() throws Exception
{
String aggSpecJson = "{\"type\": \"doubleMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
doubleMaxAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , DoubleMaxAggregatorFactory.class);
}
@Before
public void setup()
{
selector = new TestFloatColumnSelector(values);
colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class);
EasyMock.expect(colSelectorFactory.makeFloatColumnSelector("nilly")).andReturn(selector);
EasyMock.replay(colSelectorFactory);
}
@Test
public void testDoubleMaxAggregator()
{
DoubleMaxAggregator agg = (DoubleMaxAggregator) doubleMaxAggFactory.factorize(colSelectorFactory);
Assert.assertEquals("billy", agg.getName());
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
Assert.assertEquals(values[2], ((Double) agg.get()).doubleValue(), 0.0001);
Assert.assertEquals((long)values[2], agg.getLong());
Assert.assertEquals(values[2], agg.getFloat(), 0.0001);
agg.reset();
Assert.assertEquals(Double.NEGATIVE_INFINITY, (Double) agg.get(), 0.0001);
}
@Test
public void testDoubleMaxBufferAggregator()
{
DoubleMaxBufferAggregator agg = (DoubleMaxBufferAggregator) doubleMaxAggFactory.factorizeBuffered(colSelectorFactory);
ByteBuffer buffer = ByteBuffer.wrap(new byte[Doubles.BYTES]);
agg.init(buffer, 0);
aggregate(selector, agg, buffer, 0);
aggregate(selector, agg, buffer, 0);
aggregate(selector, agg, buffer, 0);
aggregate(selector, agg, buffer, 0);
Assert.assertEquals(values[2], ((Double) agg.get(buffer, 0)).doubleValue(), 0.0001);
Assert.assertEquals((long) values[2], agg.getLong(buffer, 0));
Assert.assertEquals(values[2], agg.getFloat(buffer, 0), 0.0001);
}
@Test
public void testCombine()
{
Assert.assertEquals(3.4d, ((Double) doubleMaxAggFactory.combine(1.2, 3.4)).doubleValue(), 0.0001);
}
@Test
public void testEqualsAndHashCode() throws Exception
{
DoubleMaxAggregatorFactory one = new DoubleMaxAggregatorFactory("name1", "fieldName1");
DoubleMaxAggregatorFactory oneMore = new DoubleMaxAggregatorFactory("name1", "fieldName1");
DoubleMaxAggregatorFactory two = new DoubleMaxAggregatorFactory("name2", "fieldName2");
Assert.assertEquals(one.hashCode(), oneMore.hashCode());
Assert.assertTrue(one.equals(oneMore));
Assert.assertFalse(one.equals(two));
}
@Test
public void testMaxDeserialization() throws Exception
{
String aggSpecJson = "{\"type\": \"max\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
MaxAggregatorFactory agg = new DefaultObjectMapper().readValue(aggSpecJson , MaxAggregatorFactory.class);
Assert.assertNotNull(agg);
}
private void aggregate(TestFloatColumnSelector selector, DoubleMaxAggregator agg)
{
agg.aggregate();
selector.increment();
}
private void aggregate(TestFloatColumnSelector selector, DoubleMaxBufferAggregator agg, ByteBuffer buff, int position)
{
agg.aggregate(buff, position);
selector.increment();
}
}

View File

@ -1,49 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.query.aggregation;
import org.junit.Assert;
import org.junit.Test;
/**
*/
public class DoubleMaxAggregatorTest
{
private void aggregate(TestFloatColumnSelector selector, DoubleMaxAggregator agg)
{
agg.aggregate();
selector.increment();
}
@Test
public void testAggregate() throws Exception
{
final float[] values = {0.15f, 0.27f, 0.0f, 0.93f};
final TestFloatColumnSelector selector = new TestFloatColumnSelector(values);
DoubleMaxAggregator agg = new DoubleMaxAggregator("billy", selector);
Assert.assertEquals("billy", agg.getName());
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
Assert.assertEquals(new Float(values[3]).doubleValue(), agg.get());
}
}

View File

@ -0,0 +1,133 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.query.aggregation;
import com.google.common.primitives.Doubles;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.ColumnSelectorFactory;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.nio.ByteBuffer;
/**
*/
public class DoubleMinAggregationTest
{
private DoubleMinAggregatorFactory doubleMinAggFactory;
private ColumnSelectorFactory colSelectorFactory;
private TestFloatColumnSelector selector;
private float[] values = {3.5f, 2.7f, 1.1f, 1.3f};
public DoubleMinAggregationTest() throws Exception
{
String aggSpecJson = "{\"type\": \"doubleMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
doubleMinAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , DoubleMinAggregatorFactory.class);
}
@Before
public void setup()
{
selector = new TestFloatColumnSelector(values);
colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class);
EasyMock.expect(colSelectorFactory.makeFloatColumnSelector("nilly")).andReturn(selector);
EasyMock.replay(colSelectorFactory);
}
@Test
public void testDoubleMinAggregator()
{
DoubleMinAggregator agg = (DoubleMinAggregator) doubleMinAggFactory.factorize(colSelectorFactory);
Assert.assertEquals("billy", agg.getName());
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
Assert.assertEquals(values[2], ((Double) agg.get()).doubleValue(), 0.0001);
Assert.assertEquals((long)values[2], agg.getLong());
Assert.assertEquals(values[2], agg.getFloat(), 0.0001);
agg.reset();
Assert.assertEquals(Double.POSITIVE_INFINITY, (Double) agg.get(), 0.0001);
}
@Test
public void testDoubleMinBufferAggregator()
{
DoubleMinBufferAggregator agg = (DoubleMinBufferAggregator) doubleMinAggFactory.factorizeBuffered(colSelectorFactory);
ByteBuffer buffer = ByteBuffer.wrap(new byte[Doubles.BYTES]);
agg.init(buffer, 0);
aggregate(selector, agg, buffer, 0);
aggregate(selector, agg, buffer, 0);
aggregate(selector, agg, buffer, 0);
aggregate(selector, agg, buffer, 0);
Assert.assertEquals(values[2], ((Double) agg.get(buffer, 0)).doubleValue(), 0.0001);
Assert.assertEquals((long) values[2], agg.getLong(buffer, 0));
Assert.assertEquals(values[2], agg.getFloat(buffer, 0), 0.0001);
}
@Test
public void testCombine()
{
Assert.assertEquals(1.2d, ((Double) doubleMinAggFactory.combine(1.2, 3.4)).doubleValue(), 0.0001);
}
@Test
public void testEqualsAndHashCode() throws Exception
{
DoubleMinAggregatorFactory one = new DoubleMinAggregatorFactory("name1", "fieldName1");
DoubleMinAggregatorFactory oneMore = new DoubleMinAggregatorFactory("name1", "fieldName1");
DoubleMinAggregatorFactory two = new DoubleMinAggregatorFactory("name2", "fieldName2");
Assert.assertEquals(one.hashCode(), oneMore.hashCode());
Assert.assertTrue(one.equals(oneMore));
Assert.assertFalse(one.equals(two));
}
@Test
public void testMinDeserialization() throws Exception
{
String aggSpecJson = "{\"type\": \"min\", \"name\": \"billy\", \"fieldName\": \"nilly\"}";
MinAggregatorFactory agg = new DefaultObjectMapper().readValue(aggSpecJson , MinAggregatorFactory.class);
Assert.assertNotNull(agg);
}
private void aggregate(TestFloatColumnSelector selector, DoubleMinAggregator agg)
{
agg.aggregate();
selector.increment();
}
private void aggregate(TestFloatColumnSelector selector, DoubleMinBufferAggregator agg, ByteBuffer buff, int position)
{
agg.aggregate(buff, position);
selector.increment();
}
}

View File

@ -1,49 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.query.aggregation;
import org.junit.Assert;
import org.junit.Test;
/**
*/
public class DoubleMinAggregatorTest
{
private void aggregate(TestFloatColumnSelector selector, DoubleMinAggregator agg)
{
agg.aggregate();
selector.increment();
}
@Test
public void testAggregate() throws Exception
{
final float[] values = {0.15f, 0.27f, 0.0f, 0.93f};
final TestFloatColumnSelector selector = new TestFloatColumnSelector(values);
DoubleMinAggregator agg = new DoubleMinAggregator("billy", selector);
Assert.assertEquals("billy", agg.getName());
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
aggregate(selector, agg);
Assert.assertEquals(new Float(values[2]).doubleValue(), agg.get());
}
}