提交 7a12d241 编写于 作者: V Vlad Ilyushchenko

GRIFFIN: group-by implementation

GRIFFIN: bugfix - under some circumstances SQL optimiser would produce duplicate column aliases
ALL: suppress internal sun API warnings
上级 7558838d
......@@ -64,6 +64,10 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>3.6.1</version>
<configuration>
<!-- these args suppress warning about using internal java APIs -->
<compilerArgument>-XDignore.symbol.file</compilerArgument>
<fork>true</fork>
<!-- end -->
<source>1.8</source>
<target>1.8</target>
</configuration>
......@@ -191,6 +195,9 @@
</goals>
</execution>
</executions>
<configuration>
<executable>gpg2</executable>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
......
......@@ -53,10 +53,6 @@ public class TableReaderIncrementalRecordCursor extends TableReaderRecordCursor
return false;
}
public long getLastRowId() {
return lastRowId;
}
private void seekToLast() {
if (lastRowId > -1) {
startFrom(lastRowId);
......
......@@ -320,6 +320,102 @@ public class SqlCodeGenerator {
return generateOrderBy(model, executionContext);
}
@NotNull
private RecordCursorFactory generateSampleBy(QueryModel model, SqlExecutionContext executionContext, ExpressionNode sampleByNode) throws SqlException {
final ObjList<ExpressionNode> sampleByFill = model.getSampleByFill();
final TimestampSampler timestampSampler = TimestampSamplerFactory.getInstance(sampleByNode.token, sampleByNode.position);
assert model.getNestedModel() != null;
final int fillCount = sampleByFill.size();
final RecordCursorFactory factory = generateQuery(model.getNestedModel(), executionContext);
try {
keyTypes.reset();
valueTypes.reset();
listColumnFilter.clear();
if (fillCount == 0 || fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "none")) {
return new SampleByFillNoneRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes
);
}
if (fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "prev")) {
return new SampleByFillPrevRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes
);
}
if (fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "null")) {
return new SampleByFillNullRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes
);
}
if (fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "linear")) {
return new SampleByInterpolateRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes,
entityColumnFilter
);
}
assert fillCount > 0;
return new SampleByFillValueRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
sampleByFill,
keyTypes,
valueTypes
);
} catch (SqlException | CairoException e) {
factory.close();
throw e;
}
}
private RecordCursorFactory generateSelect(QueryModel model, SqlExecutionContext executionContext) throws SqlException {
switch (model.getSelectModelType()) {
case QueryModel.SELECT_MODEL_CHOOSE:
......@@ -399,96 +495,29 @@ public class SqlCodeGenerator {
// fail fast if we cannot create timestamp sampler
final ExpressionNode sampleByNode = model.getSampleBy();
final ObjList<ExpressionNode> sampleByFill = model.getSampleByFill();
final TimestampSampler timestampSampler = TimestampSamplerFactory.getInstance(sampleByNode.token, sampleByNode.position);
if (sampleByNode != null) {
return generateSampleBy(model, executionContext, sampleByNode);
}
assert model.getNestedModel() != null;
final int fillCount = sampleByFill.size();
final RecordCursorFactory factory = generateQuery(model.getNestedModel(), executionContext);
try {
keyTypes.reset();
valueTypes.reset();
listColumnFilter.clear();
if (fillCount == 0 || fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "none")) {
return new SampleByFillNoneRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes
);
}
if (fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "prev")) {
return new SampleByFillPrevRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes
);
}
if (fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "null")) {
return new SampleByFillNullRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes
);
}
if (fillCount == 1 && Chars.equals(sampleByFill.getQuick(0).token, "linear")) {
return new SampleByInterpolateRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
keyTypes,
valueTypes,
entityColumnFilter
);
}
assert fillCount > 0;
return new SampleByFillValueRecordCursorFactory(
return new GroupByRecordCursorFactory(
configuration,
factory,
timestampSampler,
model,
listColumnFilter,
functionParser,
executionContext,
asm,
sampleByFill,
keyTypes,
valueTypes
);
} catch (SqlException | CairoException e) {
} catch (CairoException e) {
factory.close();
throw e;
}
......
......@@ -1903,7 +1903,7 @@ class SqlOptimiser {
// create virtual columns from select list
for (int i = 0, k = columns.size(); i < k; i++) {
final QueryColumn qc = columns.getQuick(i);
QueryColumn qc = columns.getQuick(i);
final boolean analytic = qc instanceof AnalyticColumn;
// fail-fast if this is an arithmetic expression where we expect analytic function
......@@ -1942,6 +1942,10 @@ class SqlOptimiser {
useAnalyticModel = true;
continue;
} else if (functionParser.isGroupBy(qc.getAst().token)) {
CharSequence alias = createColumnAlias(qc.getAlias(), groupByModel);
if (alias != qc.getAlias()) {
qc = queryColumnPool.next().of(alias, qc.getAst());
}
groupByModel.addColumn(qc);
// group-by column references might be needed when we have
// outer model supporting arithmetic such as:
......
......@@ -34,10 +34,7 @@ import com.questdb.griffin.SqlException;
import com.questdb.griffin.SqlExecutionContext;
import com.questdb.griffin.engine.functions.GroupByFunction;
import com.questdb.griffin.engine.functions.bind.BindVariableService;
import com.questdb.griffin.engine.functions.columns.*;
import com.questdb.griffin.engine.table.EmptyTableRecordCursor;
import com.questdb.griffin.model.ExpressionNode;
import com.questdb.griffin.model.QueryColumn;
import com.questdb.griffin.model.QueryModel;
import com.questdb.std.*;
import org.jetbrains.annotations.NotNull;
......@@ -68,14 +65,9 @@ public class AbstractSampleByRecordCursorFactory implements RecordCursorFactory
final int columnCount = model.getColumns().size();
final RecordMetadata metadata = base.getMetadata();
this.groupByFunctions = new ObjList<>(columnCount);
valueTypes.add(ColumnType.TIMESTAMP); // first value is always timestamp
// the goal is for fail scenarios (function does not compile) we should generate as little garbage as possible
// when we in this class we already have created a lot of objects even before we know if compilation will be successful
// first value is always timestamp
valueTypes.add(ColumnType.TIMESTAMP);
prepareGroupByFunctions(
GroupByUtils.prepareGroupByFunctions(
model,
metadata,
functionParser,
......@@ -88,7 +80,7 @@ public class AbstractSampleByRecordCursorFactory implements RecordCursorFactory
final GenericRecordMetadata groupByMetadata = new GenericRecordMetadata();
final IntIntHashMap symbolTableIndex = new IntIntHashMap();
prepareGroupByRecordFunctions(
GroupByUtils.prepareGroupByRecordFunctions(
model,
metadata,
listColumnFilter,
......@@ -97,7 +89,8 @@ public class AbstractSampleByRecordCursorFactory implements RecordCursorFactory
groupByMetadata,
keyTypes,
valueTypes,
symbolTableIndex
symbolTableIndex,
false
);
// sink will be storing record columns to map key
......@@ -117,152 +110,6 @@ public class AbstractSampleByRecordCursorFactory implements RecordCursorFactory
);
}
public static void prepareGroupByFunctions(
QueryModel model,
RecordMetadata metadata,
FunctionParser functionParser,
SqlExecutionContext executionContext,
ObjList<GroupByFunction> groupByFunctions,
ArrayColumnTypes valueTypes) throws SqlException {
final ObjList<QueryColumn> columns = model.getColumns();
for (int i = 0, n = columns.size(); i < n; i++) {
final QueryColumn column = columns.getQuick(i);
ExpressionNode node = column.getAst();
if (node.type != ExpressionNode.LITERAL) {
// this can fail
final Function function = functionParser.parseFunction(
column.getAst(),
metadata,
executionContext
);
// configure map value columns for group-by functions
// some functions may need more than one column in values
// so we have them do all the work
assert function instanceof GroupByFunction;
GroupByFunction func = (GroupByFunction) function;
func.pushValueTypes(valueTypes);
groupByFunctions.add(func);
}
}
}
public static void prepareGroupByRecordFunctions(
@NotNull QueryModel model,
RecordMetadata metadata,
@NotNull ListColumnFilter listColumnFilter,
ObjList<GroupByFunction> groupByFunctions,
ObjList<Function> recordFunctions,
GenericRecordMetadata groupByMetadata,
ArrayColumnTypes keyTypes,
ArrayColumnTypes valueTypes,
IntIntHashMap symbolTableIndex
) {
// Process group-by functions first to get the idea of
// how many map values we will have.
// Map value count is needed to calculate offsets for
// map key columns.
final int timestampIndex = metadata.getTimestampIndex();
final ObjList<QueryColumn> columns = model.getColumns();
assert timestampIndex != -1;
int keyColumnIndex = valueTypes.getColumnCount();
int valueColumnIndex = 0;
// when we have same column several times in a row
// we only add it once to map keys
int lastIndex = -1;
for (int i = 0, n = columns.size(); i < n; i++) {
final QueryColumn column = columns.getQuick(i);
final ExpressionNode node = column.getAst();
final int type;
if (node.type == ExpressionNode.LITERAL) {
// this is key
int index = metadata.getColumnIndexQuiet(node.token);
assert index != -1;
type = metadata.getColumnType(index);
if (index != timestampIndex) {
if (lastIndex != index) {
listColumnFilter.add(index);
keyTypes.add(type);
keyColumnIndex++;
lastIndex = index;
}
final Function fun;
switch (type) {
case ColumnType.BOOLEAN:
fun = new BooleanColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.BYTE:
fun = new ByteColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.SHORT:
fun = new ShortColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.INT:
fun = new IntColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.LONG:
fun = new LongColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.FLOAT:
fun = new FloatColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.DOUBLE:
fun = new DoubleColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.STRING:
fun = new StrColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.SYMBOL:
symbolTableIndex.put(i, index);
fun = new MapSymbolColumn(node.position, keyColumnIndex - 1, i);
break;
case ColumnType.DATE:
fun = new DateColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.TIMESTAMP:
fun = new TimestampColumn(node.position, keyColumnIndex - 1);
break;
default:
fun = new BinColumn(node.position, keyColumnIndex - 1);
break;
}
recordFunctions.add(fun);
} else {
// set this function to null, cursor will replace it with an instance class
// timestamp function returns value of class member which makes it impossible
// to create these columns in advance of cursor instantiation
recordFunctions.add(null);
if (groupByMetadata.getTimestampIndex() == -1) {
groupByMetadata.setTimestampIndex(i);
}
assert type == ColumnType.TIMESTAMP;
}
} else {
// add group-by function as a record function as well
// so it can produce column values
final GroupByFunction groupByFunction = groupByFunctions.getQuick(valueColumnIndex++);
recordFunctions.add(groupByFunction);
type = groupByFunction.getType();
}
// and finish with populating metadata for this factory
groupByMetadata.add(new TableColumnMetadata(
Chars.toString(column.getName()),
type
));
}
}
@Override
public void close() {
// todo: test that functions are indeed being freed
......
/*******************************************************************************
* ___ _ ____ ____
* / _ \ _ _ ___ ___| |_| _ \| __ )
* | | | | | | |/ _ \/ __| __| | | | _ \
* | |_| | |_| | __/\__ \ |_| |_| | |_) |
* \__\_\\__,_|\___||___/\__|____/|____/
*
* Copyright (C) 2014-2018 Appsicle
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License, version 3,
* as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*
******************************************************************************/
package com.questdb.griffin.engine.groupby;
import com.questdb.cairo.*;
import com.questdb.cairo.map.Map;
import com.questdb.cairo.map.MapFactory;
import com.questdb.cairo.map.MapKey;
import com.questdb.cairo.map.MapValue;
import com.questdb.cairo.sql.*;
import com.questdb.griffin.FunctionParser;
import com.questdb.griffin.SqlException;
import com.questdb.griffin.SqlExecutionContext;
import com.questdb.griffin.engine.functions.GroupByFunction;
import com.questdb.griffin.engine.functions.bind.BindVariableService;
import com.questdb.griffin.model.QueryModel;
import com.questdb.std.*;
import org.jetbrains.annotations.NotNull;
public class GroupByRecordCursorFactory implements RecordCursorFactory {
protected final RecordCursorFactory base;
private final Map dataMap;
private final GroupByRecordCursor cursor;
private final ObjList<Function> recordFunctions;
private final ObjList<GroupByFunction> groupByFunctions;
private final RecordSink mapSink;
// this sink is used to copy recordKeyMap keys to dataMap
private final RecordMetadata metadata;
public GroupByRecordCursorFactory(
CairoConfiguration configuration,
RecordCursorFactory base,
@Transient @NotNull QueryModel model,
@Transient @NotNull ListColumnFilter listColumnFilter,
@Transient @NotNull FunctionParser functionParser,
@Transient @NotNull SqlExecutionContext executionContext,
@Transient @NotNull BytecodeAssembler asm,
@Transient @NotNull ArrayColumnTypes keyTypes,
@Transient @NotNull ArrayColumnTypes valueTypes
) throws SqlException {
final int columnCount = model.getColumns().size();
final RecordMetadata metadata = base.getMetadata();
this.groupByFunctions = new ObjList<>(columnCount);
GroupByUtils.prepareGroupByFunctions(
model,
metadata,
functionParser,
executionContext,
groupByFunctions,
valueTypes
);
this.recordFunctions = new ObjList<>(columnCount);
final GenericRecordMetadata groupByMetadata = new GenericRecordMetadata();
final IntIntHashMap symbolTableIndex = new IntIntHashMap();
GroupByUtils.prepareGroupByRecordFunctions(
model,
metadata,
listColumnFilter,
groupByFunctions,
recordFunctions,
groupByMetadata,
keyTypes,
valueTypes,
symbolTableIndex,
true
);
// sink will be storing record columns to map key
this.mapSink = RecordSinkFactory.getInstance(asm, metadata, listColumnFilter, false);
this.dataMap = MapFactory.createMap(configuration, keyTypes, valueTypes);
this.base = base;
this.metadata = groupByMetadata;
this.cursor = new GroupByRecordCursor(recordFunctions, symbolTableIndex);
}
@Override
public void close() {
// todo: test that functions are indeed being freed
for (int i = 0, n = recordFunctions.size(); i < n; i++) {
recordFunctions.getQuick(i).close();
}
dataMap.close();
}
@Override
public RecordCursor getCursor(BindVariableService bindVariableService) {
dataMap.clear();
final RecordCursor baseCursor = base.getCursor(bindVariableService);
try {
final Record baseRecord = baseCursor.getRecord();
final int n = groupByFunctions.size();
while (baseCursor.hasNext()) {
final MapKey key = dataMap.withKey();
mapSink.copy(baseRecord, key);
MapValue value = key.createValue();
if (value.isNew()) {
for (int i = 0; i < n; i++) {
groupByFunctions.getQuick(i).computeFirst(value, baseRecord);
}
} else {
for (int i = 0; i < n; i++) {
groupByFunctions.getQuick(i).computeNext(value, baseRecord);
}
}
}
return initFunctionsAndCursor(bindVariableService, dataMap.getCursor(), baseCursor);
} catch (CairoException e) {
baseCursor.close();
//todo: free other things
throw e;
}
}
@Override
public RecordMetadata getMetadata() {
return metadata;
}
@Override
public boolean isRandomAccessCursor() {
return true;
}
@NotNull
protected RecordCursor initFunctionsAndCursor(BindVariableService bindVariableService, RecordCursor mapCursor, RecordCursor baseCursor) {
cursor.of(mapCursor, baseCursor);
// init all record function for this cursor, in case functions require metadata and/or symbol tables
for (int i = 0, m = recordFunctions.size(); i < m; i++) {
recordFunctions.getQuick(i).init(cursor, bindVariableService);
}
return cursor;
}
private static class GroupByRecordCursor implements RecordCursor {
private final VirtualRecord functionRecord;
private final IntIntHashMap symbolTableIndex;
private RecordCursor mapCursor;
private RecordCursor baseCursor;
public GroupByRecordCursor(ObjList<Function> functions, IntIntHashMap symbolTableIndex) {
this.functionRecord = new VirtualRecord(functions);
this.symbolTableIndex = symbolTableIndex;
}
@Override
public void close() {
Misc.free(mapCursor);
Misc.free(baseCursor);
}
@Override
public Record getRecord() {
return functionRecord;
}
@Override
public SymbolTable getSymbolTable(int columnIndex) {
return baseCursor.getSymbolTable(symbolTableIndex.get(columnIndex));
}
@Override
public boolean hasNext() {
return mapCursor.hasNext();
}
@Override
public Record newRecord() {
VirtualRecord record = new VirtualRecord(functionRecord.getFunctions());
record.of(mapCursor.newRecord());
return record;
}
@Override
public void recordAt(Record record, long atRowId) {
assert record instanceof VirtualRecord;
mapCursor.recordAt(((VirtualRecord) record).getBaseRecord(), atRowId);
}
@Override
public void recordAt(long rowId) {
mapCursor.recordAt(functionRecord.getBaseRecord(), rowId);
}
@Override
public void toTop() {
mapCursor.toTop();
}
public void of(RecordCursor mapCursor, RecordCursor baseCursor) {
this.mapCursor = mapCursor;
this.baseCursor = baseCursor;
functionRecord.of(mapCursor.getRecord());
}
}
}
/*******************************************************************************
* ___ _ ____ ____
* / _ \ _ _ ___ ___| |_| _ \| __ )
* | | | | | | |/ _ \/ __| __| | | | _ \
* | |_| | |_| | __/\__ \ |_| |_| | |_) |
* \__\_\\__,_|\___||___/\__|____/|____/
*
* Copyright (C) 2014-2018 Appsicle
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License, version 3,
* as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU Affero General Public License for more details.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*
******************************************************************************/
package com.questdb.griffin.engine.groupby;
import com.questdb.cairo.*;
import com.questdb.cairo.sql.Function;
import com.questdb.cairo.sql.RecordMetadata;
import com.questdb.griffin.FunctionParser;
import com.questdb.griffin.SqlException;
import com.questdb.griffin.SqlExecutionContext;
import com.questdb.griffin.engine.functions.GroupByFunction;
import com.questdb.griffin.engine.functions.columns.*;
import com.questdb.griffin.model.ExpressionNode;
import com.questdb.griffin.model.QueryColumn;
import com.questdb.griffin.model.QueryModel;
import com.questdb.std.Chars;
import com.questdb.std.IntIntHashMap;
import com.questdb.std.ObjList;
import org.jetbrains.annotations.NotNull;
class GroupByUtils {
static void prepareGroupByFunctions(
QueryModel model,
RecordMetadata metadata,
FunctionParser functionParser,
SqlExecutionContext executionContext,
ObjList<GroupByFunction> groupByFunctions,
ArrayColumnTypes valueTypes) throws SqlException {
final ObjList<QueryColumn> columns = model.getColumns();
for (int i = 0, n = columns.size(); i < n; i++) {
final QueryColumn column = columns.getQuick(i);
ExpressionNode node = column.getAst();
if (node.type != ExpressionNode.LITERAL) {
// this can fail
final Function function = functionParser.parseFunction(
column.getAst(),
metadata,
executionContext
);
// configure map value columns for group-by functions
// some functions may need more than one column in values
// so we have them do all the work
assert function instanceof GroupByFunction;
GroupByFunction func = (GroupByFunction) function;
func.pushValueTypes(valueTypes);
groupByFunctions.add(func);
}
}
}
static void prepareGroupByRecordFunctions(
@NotNull QueryModel model,
RecordMetadata metadata,
@NotNull ListColumnFilter listColumnFilter,
ObjList<GroupByFunction> groupByFunctions,
ObjList<Function> recordFunctions,
GenericRecordMetadata groupByMetadata,
ArrayColumnTypes keyTypes,
ArrayColumnTypes valueTypes,
IntIntHashMap symbolTableIndex,
boolean timestampUnimportant
) {
// Process group-by functions first to get the idea of
// how many map values we will have.
// Map value count is needed to calculate offsets for
// map key columns.
final int timestampIndex = metadata.getTimestampIndex();
final ObjList<QueryColumn> columns = model.getColumns();
assert timestampIndex != -1;
int keyColumnIndex = valueTypes.getColumnCount();
int valueColumnIndex = 0;
// when we have same column several times in a row
// we only add it once to map keys
int lastIndex = -1;
for (int i = 0, n = columns.size(); i < n; i++) {
final QueryColumn column = columns.getQuick(i);
final ExpressionNode node = column.getAst();
final int type;
if (node.type == ExpressionNode.LITERAL) {
// this is key
int index = metadata.getColumnIndexQuiet(node.token);
assert index != -1;
type = metadata.getColumnType(index);
if (index != timestampIndex || timestampUnimportant) {
if (lastIndex != index) {
listColumnFilter.add(index);
keyTypes.add(type);
keyColumnIndex++;
lastIndex = index;
}
final Function fun;
switch (type) {
case ColumnType.BOOLEAN:
fun = new BooleanColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.BYTE:
fun = new ByteColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.SHORT:
fun = new ShortColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.INT:
fun = new IntColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.LONG:
fun = new LongColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.FLOAT:
fun = new FloatColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.DOUBLE:
fun = new DoubleColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.STRING:
fun = new StrColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.SYMBOL:
symbolTableIndex.put(i, index);
fun = new MapSymbolColumn(node.position, keyColumnIndex - 1, i);
break;
case ColumnType.DATE:
fun = new DateColumn(node.position, keyColumnIndex - 1);
break;
case ColumnType.TIMESTAMP:
fun = new TimestampColumn(node.position, keyColumnIndex - 1);
break;
default:
fun = new BinColumn(node.position, keyColumnIndex - 1);
break;
}
recordFunctions.add(fun);
} else {
// set this function to null, cursor will replace it with an instance class
// timestamp function returns value of class member which makes it impossible
// to create these columns in advance of cursor instantiation
recordFunctions.add(null);
if (groupByMetadata.getTimestampIndex() == -1) {
groupByMetadata.setTimestampIndex(i);
}
assert type == ColumnType.TIMESTAMP;
}
} else {
// add group-by function as a record function as well
// so it can produce column values
final GroupByFunction groupByFunction = groupByFunctions.getQuick(valueColumnIndex++);
recordFunctions.add(groupByFunction);
type = groupByFunction.getType();
}
// and finish with populating metadata for this factory
groupByMetadata.add(new TableColumnMetadata(
Chars.toString(column.getName()),
type
));
}
}
}
......@@ -77,7 +77,7 @@ public class SampleByInterpolateRecordCursorFactory implements RecordCursorFacto
this.groupByFunctions = new ObjList<>(columnCount);
valueTypes.add(ColumnType.BYTE); // gap flag
AbstractSampleByRecordCursorFactory.prepareGroupByFunctions(
GroupByUtils.prepareGroupByFunctions(
model,
metadata,
functionParser,
......@@ -90,7 +90,7 @@ public class SampleByInterpolateRecordCursorFactory implements RecordCursorFacto
final GenericRecordMetadata groupByMetadata = new GenericRecordMetadata();
final IntIntHashMap symbolTableIndex = new IntIntHashMap();
AbstractSampleByRecordCursorFactory.prepareGroupByRecordFunctions(
GroupByUtils.prepareGroupByRecordFunctions(
model,
metadata,
listColumnFilter,
......@@ -99,7 +99,8 @@ public class SampleByInterpolateRecordCursorFactory implements RecordCursorFacto
groupByMetadata,
keyTypes,
valueTypes,
symbolTableIndex
symbolTableIndex,
false
);
this.storeYFunctions = new ObjList<>(columnCount);
......
......@@ -50,6 +50,56 @@ public class SampleByTest extends AbstractGriffinTest {
"Invalid column: c");
}
@Test
public void testGroupByAllTypes() throws Exception {
assertQuery("b\tsum\tsum1\tsum2\tsum3\tsum4\tsum5\n" +
"HYRX\t108.4198\t129.399112218477\t2127224767\t95\t-8329\t1696566079386694074\n" +
"\t680.7651\t771.092262202840\t2135522192\t77\t815\t-5259855777509188759\n" +
"CPSW\t101.2276\t111.113584037391\t-1727443926\t33\t-22282\t7594916031131877487\n" +
"PEHN\t104.2904\t100.877261378303\t-940643167\t18\t17565\t-4882690809235649274\n" +
"RXGZ\t96.4029\t42.020442539326\t712702244\t46\t22661\t2762535352290012031\n",
"select b, sum(a), sum(c), sum(d), sum(e), sum(f), sum(g) from x",
"create table x as " +
"(" +
"select" +
" rnd_float(0)*100 a," +
" rnd_symbol(5,4,4,1) b," +
" rnd_double(0)*100 c," +
" abs(rnd_int()) d," +
" rnd_byte(2, 50) e," +
" abs(rnd_short()) f," +
" abs(rnd_long()) g," +
" timestamp_sequence(to_timestamp(172800000000), 3600000000) k" +
" from" +
" long_sequence(20)" +
") timestamp(k) partition by NONE",
null,
"insert into x select * from (" +
"select" +
" rnd_float(0)*100 a," +
" rnd_symbol(5,4,4,1) b," +
" rnd_double(0)*100 c," +
" abs(rnd_int()) d," +
" rnd_byte(2, 50) e," +
" abs(rnd_short()) f," +
" abs(rnd_long()) g," +
" timestamp_sequence(to_timestamp(277200000000), 3600000000) k" +
" from" +
" long_sequence(5)" +
") timestamp(k)",
"b\tsum\tsum1\tsum2\tsum3\tsum4\tsum5\n" +
"HYRX\t108.4198\t129.399112218477\t2127224767\t95\t-8329\t1696566079386694074\n" +
"\t779.3558\t869.932373151714\t-247384018\t107\t18639\t3597805051091659961\n" +
"CPSW\t101.2276\t111.113584037391\t-1727443926\t33\t-22282\t7594916031131877487\n" +
"PEHN\t104.2904\t100.877261378303\t-940643167\t18\t17565\t-4882690809235649274\n" +
"RXGZ\t96.4029\t42.020442539326\t712702244\t46\t22661\t2762535352290012031\n" +
"ZGHW\t50.2589\t38.422543844715\t597366062\t21\t23702\t7037372650941669660\n" +
"LOPJ\t76.6815\t5.158459929274\t1920398380\t38\t16628\t3527911398466283309\n" +
"VDKF\t4.3606\t35.681110212277\t503883303\t38\t10895\t7202923278768687325\n" +
"OXPK\t45.9207\t76.062526341246\t2043541236\t21\t19278\t1832315370633201942\n",
true);
}
@Test
public void testSampleBadFunction() throws Exception {
assertFailure(
......
......@@ -1152,6 +1152,42 @@ public class SqlParserTest extends AbstractGriffinTest {
);
}
@Test
public void testDuplicateColumnGroupBy() throws SqlException {
assertQuery(
"select-group-by b, sum(a) sum, k1, k1 k from (select-choose b, a, k k1 from (x y timestamp (timestamp))) sample by 3h",
"select b, sum(a), k k1, k from x y sample by 3h",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testDuplicateColumnsBasicSelect() throws SqlException {
assertQuery(
"select-choose b, a, k1, k1 k from (select-choose b, a, k k1 from (x timestamp (timestamp)))",
"select b, a, k k1, k from x",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testDuplicateColumnsVirtualAndGroupBySelect() throws SqlException {
assertQuery(
"select-group-by sum(b + a) sum, column, k1, k1 k from (select-virtual a, b, a + b column, k1 from (select-choose a, b, k k1 from (x timestamp (timestamp)))) sample by 1m",
"select sum(b+a), a+b, k k1, k from x sample by 1m",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testDuplicateColumnsVirtualSelect() throws SqlException {
assertQuery(
"select-choose column, k1, k1 k from (select-virtual b + a column, k1 from (select-choose a, b, k k1 from (x timestamp (timestamp))))",
"select b+a, k k1, k from x",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testDuplicateTables() throws Exception {
assertQuery(
......@@ -2980,54 +3016,6 @@ public class SqlParserTest extends AbstractGriffinTest {
);
}
@Test
public void testDuplicateColumnGroupBy() throws SqlException {
assertQuery(
"select-group-by b, sum(a) sum, k1, k1 k from (select-choose b, a, k k1 from (x y timestamp (timestamp))) sample by 3h",
"select b, sum(a), k k1, k from x y sample by 3h",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testDuplicateColumnsBasicSelect() throws SqlException {
assertQuery(
"select-choose b, a, k1, k1 k from (select-choose b, a, k k1 from (x timestamp (timestamp)))",
"select b, a, k k1, k from x",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testDuplicateColumnsVirtualAndGroupBySelect() throws SqlException {
assertQuery(
"select-group-by sum(b + a) sum, column, k1, k1 k from (select-virtual a, b, a + b column, k1 from (select-choose a, b, k k1 from (x timestamp (timestamp)))) sample by 1m",
"select sum(b+a), a+b, k k1, k from x sample by 1m",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testDuplicateColumnsVirtualSelect() throws SqlException {
assertQuery(
"select-choose column, k1, k1 k from (select-virtual b + a column, k1 from (select-choose a, b, k k1 from (x timestamp (timestamp))))",
"select b+a, k k1, k from x",
modelOf("x").col("a", ColumnType.DOUBLE).col("b", ColumnType.SYMBOL).col("k", ColumnType.TIMESTAMP).timestamp()
);
}
@Test
public void testSampleByFillMin() throws SqlException {
assertQuery(
"select-group-by a, sum(b) b from (tab timestamp (t)) sample by 10m fill(mid)",
"select a,sum(b) b from tab timestamp(t) sample by 10m fill(mid)",
modelOf("tab")
.col("a", ColumnType.INT)
.col("b", ColumnType.INT)
.col("t", ColumnType.TIMESTAMP)
);
}
@Test
public void testOuterJoin() throws Exception {
assertQuery(
......@@ -3142,6 +3130,53 @@ public class SqlParserTest extends AbstractGriffinTest {
);
}
@Test
public void testSampleByAlreadySelected() throws Exception {
assertQuery(
"select-group-by x, sum(y) sum from (tab timestamp (x)) sample by 2m",
"select x,sum(y) from tab timestamp(x) sample by 2m",
modelOf("tab")
.col("x", ColumnType.TIMESTAMP)
.col("y", ColumnType.INT)
);
}
@Test
public void testSampleByAltTimestamp() throws Exception {
assertQuery(
"select-group-by x, sum(y) sum from (tab timestamp (t)) sample by 2m",
"select x,sum(y) from tab timestamp(t) sample by 2m",
modelOf("tab")
.col("x", ColumnType.INT)
.col("y", ColumnType.INT)
.col("t", ColumnType.TIMESTAMP)
);
}
@Test
public void testSampleByFillList() throws SqlException {
assertQuery(
"select-group-by a, sum(b) b from (tab timestamp (t)) sample by 10m fill(21.1,22,null,98)",
"select a,sum(b) b from tab timestamp(t) sample by 10m fill(21.1,22,null,98)",
modelOf("tab")
.col("a", ColumnType.INT)
.col("b", ColumnType.INT)
.col("t", ColumnType.TIMESTAMP)
);
}
@Test
public void testSampleByFillMin() throws SqlException {
assertQuery(
"select-group-by a, sum(b) b from (tab timestamp (t)) sample by 10m fill(mid)",
"select a,sum(b) b from tab timestamp(t) sample by 10m fill(mid)",
modelOf("tab")
.col("a", ColumnType.INT)
.col("b", ColumnType.INT)
.col("t", ColumnType.TIMESTAMP)
);
}
@Test
public void testSampleByFillMissingCloseBrace() {
assertSyntaxError(
......@@ -3193,41 +3228,6 @@ public class SqlParserTest extends AbstractGriffinTest {
);
}
@Test
public void testSampleByFillList() throws SqlException {
assertQuery(
"select-group-by a, sum(b) b from (tab timestamp (t)) sample by 10m fill(21.1,22,null,98)",
"select a,sum(b) b from tab timestamp(t) sample by 10m fill(21.1,22,null,98)",
modelOf("tab")
.col("a", ColumnType.INT)
.col("b", ColumnType.INT)
.col("t", ColumnType.TIMESTAMP)
);
}
@Test
public void testSampleByAlreadySelected() throws Exception {
assertQuery(
"select-group-by x, sum(y) sum from (tab timestamp (x)) sample by 2m",
"select x,sum(y) from tab timestamp(x) sample by 2m",
modelOf("tab")
.col("x", ColumnType.TIMESTAMP)
.col("y", ColumnType.INT)
);
}
@Test
public void testSampleByAltTimestamp() throws Exception {
assertQuery(
"select-group-by x, sum(y) sum from (tab timestamp (t)) sample by 2m",
"select x,sum(y) from tab timestamp(t) sample by 2m",
modelOf("tab")
.col("x", ColumnType.INT)
.col("y", ColumnType.INT)
.col("t", ColumnType.TIMESTAMP)
);
}
@Test
public void testSampleByIncorrectPlacement() {
assertSyntaxError(
......@@ -3332,6 +3332,16 @@ public class SqlParserTest extends AbstractGriffinTest {
);
}
@Test
public void testSelectGroupByArithmetic() throws SqlException {
assertQuery("select-virtual sum + 10 column, sum1 from (select-group-by sum(x) sum, sum(y) sum1 from (tab))",
"select sum(x)+10, sum(y) from tab",
modelOf("tab")
.col("x", ColumnType.INT)
.col("y", ColumnType.INT)
);
}
@Test
public void testSelectMissingExpression() {
assertSyntaxError(
......@@ -3355,15 +3365,6 @@ public class SqlParserTest extends AbstractGriffinTest {
assertSyntaxError("select ", 7, "column expected");
}
@Test
public void testSelectSelectColumn() {
assertSyntaxError(
"select a, select from tab",
17,
"reserved name"
);
}
@Test
public void testSelectPlainColumns() throws Exception {
assertQuery(
......@@ -3373,6 +3374,15 @@ public class SqlParserTest extends AbstractGriffinTest {
);
}
@Test
public void testSelectSelectColumn() {
assertSyntaxError(
"select a, select from tab",
17,
"reserved name"
);
}
@Test
public void testSelectSingleExpression() throws Exception {
assertQuery(
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册