未验证 提交 59412c5c 编写于 作者: V Vlad Ilyushchenko 提交者: GitHub

feat(griffin): time interval search calculates row count more efficiently (#567)

Added check for RecordCursor.size() regression
上级 002f0141
......@@ -31,6 +31,8 @@ import io.questdb.std.LongList;
import io.questdb.std.Transient;
public abstract class AbstractIntervalDataFrameCursor implements DataFrameCursor {
static final int SCAN_UP = -1;
static final int SCAN_DOWN = 1;
protected final LongList intervals;
protected final IntervalDataFrame dataFrame = new IntervalDataFrame();
protected final int timestampIndex;
......@@ -43,41 +45,19 @@ public abstract class AbstractIntervalDataFrameCursor implements DataFrameCursor
// than one searches to be performed we can use this variable to avoid
// searching partition from top every time
protected long partitionLimit;
protected long sizeSoFar = 0;
protected long size = -1;
private int initialIntervalsLo;
private int initialIntervalsHi;
private int initialPartitionLo;
private int initialPartitionHi;
static final int SCAN_UP = -1;
static final int SCAN_DOWN = 1;
public AbstractIntervalDataFrameCursor(@Transient LongList intervals, int timestampIndex) {
assert timestampIndex > -1;
this.intervals = new LongList(intervals);
this.timestampIndex = timestampIndex;
}
protected static long search(ReadOnlyColumn column, long value, long low, long high, int increment) {
while (low < high) {
long mid = (low + high - 1) >>> 1;
long midVal = column.getLong(mid * 8);
if (midVal < value)
low = mid + 1;
else if (midVal > value)
high = mid;
else {
// In case of multiple equal values, find the first
mid += increment;
while (mid > 0 && mid < high && midVal == column.getLong(mid * 8)) {
mid += increment;
}
return mid - increment;
}
}
return -(low + 1);
}
@Override
public void close() {
if (reader != null) {
......@@ -91,6 +71,15 @@ public abstract class AbstractIntervalDataFrameCursor implements DataFrameCursor
return reader.getSymbolMapReader(columnIndex);
}
@Override
public void toTop() {
intervalsLo = initialIntervalsLo;
intervalsHi = initialIntervalsHi;
partitionLo = initialPartitionLo;
partitionHi = initialPartitionHi;
sizeSoFar = 0;
}
@Override
public TableReader getTableReader() {
return reader;
......@@ -106,11 +95,8 @@ public abstract class AbstractIntervalDataFrameCursor implements DataFrameCursor
}
@Override
public void toTop() {
intervalsLo = initialIntervalsLo;
intervalsHi = initialIntervalsHi;
partitionLo = initialPartitionLo;
partitionHi = initialPartitionHi;
public long size() {
return size > -1 ? size : computeSize();
}
public void of(TableReader reader) {
......@@ -118,7 +104,29 @@ public abstract class AbstractIntervalDataFrameCursor implements DataFrameCursor
calculateRanges();
}
protected static long search(ReadOnlyColumn column, long value, long low, long high, int increment) {
while (low < high) {
long mid = (low + high - 1) >>> 1;
long midVal = column.getLong(mid * 8);
if (midVal < value)
low = mid + 1;
else if (midVal > value)
high = mid;
else {
// In case of multiple equal values, find the first
mid += increment;
while (mid > 0 && mid < high && midVal == column.getLong(mid * 8)) {
mid += increment;
}
return mid - increment;
}
}
return -(low + 1);
}
private void calculateRanges() {
size = -1;
if (intervals.size() > 0) {
if (reader.getPartitionedBy() == PartitionBy.NONE) {
initialIntervalsLo = 0;
......@@ -135,6 +143,90 @@ public abstract class AbstractIntervalDataFrameCursor implements DataFrameCursor
}
}
private long computeSize() {
int intervalsLo = this.intervalsLo;
int intervalsHi = this.intervalsHi;
int partitionLo = this.partitionLo;
int partitionHi = this.partitionHi;
long partitionLimit = this.partitionLimit;
long size = this.sizeSoFar;
while (intervalsLo < intervalsHi && partitionLo < partitionHi) {
// We don't need to worry about column tops and null column because we
// are working with timestamp. Timestamp column cannot be added to existing table.
long rowCount = reader.openPartition(partitionLo);
if (rowCount > 0) {
final ReadOnlyColumn column = reader.getColumn(TableReader.getPrimaryColumnIndex(reader.getColumnBase(partitionLo), timestampIndex));
final long intervalLo = intervals.getQuick(intervalsLo * 2);
final long intervalHi = intervals.getQuick(intervalsLo * 2 + 1);
final long partitionTimestampLo = column.getLong(0);
// interval is wholly above partition, skip interval
if (partitionTimestampLo > intervalHi) {
intervalsLo++;
continue;
}
final long partitionTimestampHi = column.getLong((rowCount - 1) * 8);
// interval is wholly below partition, skip partition
if (partitionTimestampHi < intervalLo) {
partitionLimit = 0;
partitionLo++;
continue;
}
// calculate intersection
long lo;
if (partitionTimestampLo == intervalLo) {
lo = 0;
} else {
lo = search(column, intervalLo, partitionLimit, rowCount, AbstractIntervalDataFrameCursor.SCAN_UP);
if (lo < 0) {
lo = -lo - 1;
}
}
long hi = search(column, intervalHi, lo, rowCount, AbstractIntervalDataFrameCursor.SCAN_DOWN);
if (hi < 0) {
hi = -hi - 1;
} else {
// We have direct hit. Interval is inclusive of edges and we have to
// bump to high bound because it is non-inclusive
hi++;
}
if (lo < hi) {
size += (hi - lo);
// we do have whole partition of fragment?
if (hi == rowCount) {
// whole partition, will need to skip to next one
partitionLimit = 0;
partitionLo++;
} else {
// only fragment, need to skip to next interval
partitionLimit = hi;
intervalsLo++;
}
continue;
}
// interval yielded empty data frame
partitionLimit = hi;
intervalsLo++;
} else {
// partition was empty, just skip to next
partitionLo++;
}
}
return this.size = size;
}
private void cullIntervals() {
int intervalsLo = intervals.binarySearch(reader.getMinTimestamp());
......
......@@ -96,6 +96,8 @@ public class IntervalBwdDataFrameCursor extends AbstractIntervalDataFrameCursor
dataFrame.rowLo = lo;
dataFrame.rowHi = hi;
sizeSoFar += hi - lo;
// we do have whole partition of fragment?
if (lo == 0) {
// whole partition, will need to skip to next one
......@@ -120,11 +122,6 @@ public class IntervalBwdDataFrameCursor extends AbstractIntervalDataFrameCursor
return null;
}
@Override
public long size() {
return -1;
}
@Override
public void toTop() {
super.toTop();
......
......@@ -97,6 +97,7 @@ public class IntervalFwdDataFrameCursor extends AbstractIntervalDataFrameCursor
dataFrame.partitionIndex = partitionLo;
dataFrame.rowLo = lo;
dataFrame.rowHi = hi;
sizeSoFar += (hi - lo);
// we do have whole partition of fragment?
if (hi == rowCount) {
......@@ -122,11 +123,6 @@ public class IntervalFwdDataFrameCursor extends AbstractIntervalDataFrameCursor
return null;
}
@Override
public long size() {
return -1;
}
@Override
public void toTop() {
super.toTop();
......
......@@ -368,7 +368,6 @@ public class PGConnectionContext implements IOContext, Mutable {
return this;
}
@SuppressWarnings("unused")
public void setBooleanBindVariable(int index, long address, int valueLen) throws SqlException {
if (valueLen != 4 && valueLen != 5) {
throw SqlException.$(0, "bad value for BOOLEAN parameter [index=").put(index).put(", valueLen=").put(valueLen).put(']');
......@@ -458,7 +457,6 @@ public class PGConnectionContext implements IOContext, Mutable {
}
}
@SuppressWarnings("unused")
public void setNoopBindVariable(int index, long address, int valueLen) {
}
......@@ -1198,7 +1196,7 @@ public class PGConnectionContext implements IOContext, Mutable {
@Transient AssociativeCache<Object> factoryCache,
@Transient CharSequenceObjHashMap<NamedStatementWrapper> namedStatementMap,
@Transient ObjList<BindVariableSetter> bindVariableSetters
) throws BadProtocolException, SqlException, PeerDisconnectedException, PeerIsSlowToReadException {
) throws BadProtocolException, SqlException {
// 'Parse'
// this appears to be the execution side - we must at least return 'RowDescription'
// possibly more, check QueryExecutionImpl.processResults() in PG driver for more info
......@@ -1566,12 +1564,11 @@ public class PGConnectionContext implements IOContext, Mutable {
}
responseAsciiSink.putLen(addr);
transientCopyBuffer = Unsafe.malloc(1024 * 1024);
send();
} else {
prepareError(SqlException.$(0, "table '").put(textLoader.getTableName()).put("' does not exist"));
prepareReadyForQuery(responseAsciiSink);
send();
}
send();
}
private void prepareRowDescription() {
......
/*******************************************************************************
* ___ _ ____ ____
* / _ \ _ _ ___ ___| |_| _ \| __ )
* | | | | | | |/ _ \/ __| __| | | | _ \
* | |_| | |_| | __/\__ \ |_| |_| | |_) |
* \__\_\\__,_|\___||___/\__|____/|____/
*
* Copyright (c) 2014-2019 Appsicle
* Copyright (c) 2019-2020 QuestDB
*
* 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.questdb.griffin.engine;
import io.questdb.cairo.sql.NoRandomAccessRecordCursor;
import io.questdb.cairo.sql.Record;
import io.questdb.cairo.sql.VirtualRecordNoRowid;
import io.questdb.std.ObjList;
final public class EmptyTableNoSizeRecordCursor implements NoRandomAccessRecordCursor {
public static final EmptyTableNoSizeRecordCursor INSTANCE = new EmptyTableNoSizeRecordCursor();
private final Record record = new VirtualRecordNoRowid(new ObjList<>());
@Override
public void close() {
}
@Override
public Record getRecord() {
return record;
}
@Override
public boolean hasNext() {
return false;
}
@Override
public void toTop() {
}
@Override
public long size() {
return -1;
}
}
......@@ -170,7 +170,7 @@ public class LimitRecordCursorFactory extends AbstractRecordCursorFactory {
size = limit;
base.toTop();
} else {
limit = hi - lo;
limit = Math.max(0, hi - lo);
size = limit;
}
......
......@@ -30,7 +30,7 @@ import io.questdb.cairo.sql.RecordCursor;
import io.questdb.cairo.sql.RecordCursorFactory;
import io.questdb.cairo.sql.RecordMetadata;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.std.Misc;
import io.questdb.std.ObjList;
......@@ -83,7 +83,7 @@ public class SampleByFillNoneNotKeyedRecordCursorFactory implements RecordCursor
return initFunctionsAndCursor(executionContext, baseCursor);
}
baseCursor.close();
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
}
@Override
......
......@@ -32,7 +32,7 @@ import io.questdb.cairo.sql.RecordCursor;
import io.questdb.cairo.sql.RecordCursorFactory;
import io.questdb.cairo.sql.RecordMetadata;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.std.BytecodeAssembler;
import io.questdb.std.Misc;
......@@ -99,7 +99,7 @@ public class SampleByFillNoneRecordCursorFactory implements RecordCursorFactory
return initFunctionsAndCursor(executionContext, baseCursor);
}
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
} catch (CairoException ex) {
baseCursor.close();
throw ex;
......
......@@ -31,7 +31,7 @@ import io.questdb.cairo.sql.RecordCursorFactory;
import io.questdb.cairo.sql.RecordMetadata;
import io.questdb.griffin.SqlException;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.std.Misc;
import io.questdb.std.ObjList;
......@@ -85,7 +85,7 @@ public class SampleByFillNullNotKeyedRecordCursorFactory implements RecordCursor
return initFunctionsAndCursor(executionContext, baseCursor);
}
Misc.free(baseCursor);
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
}
@Override
......
......@@ -33,7 +33,7 @@ import io.questdb.cairo.sql.*;
import io.questdb.griffin.SqlException;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.SqlExecutionInterruptor;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.griffin.engine.functions.constants.*;
import io.questdb.std.*;
......@@ -164,7 +164,7 @@ public class SampleByFillNullRecordCursorFactory implements RecordCursorFactory
// empty map? this means that base cursor was empty
if (map.size() == 0) {
baseCursor.close();
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
}
// because we pass base cursor twice we have to go back to top
......
......@@ -30,7 +30,7 @@ import io.questdb.cairo.sql.RecordCursor;
import io.questdb.cairo.sql.RecordCursorFactory;
import io.questdb.cairo.sql.RecordMetadata;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.std.Misc;
import io.questdb.std.ObjList;
......@@ -82,7 +82,7 @@ public class SampleByFillPrevNotKeyedRecordCursorFactory implements RecordCursor
return initFunctionsAndCursor(executionContext, baseCursor);
}
baseCursor.close();
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
}
@Override
......
......@@ -32,7 +32,7 @@ import io.questdb.cairo.map.MapValue;
import io.questdb.cairo.sql.*;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.SqlExecutionInterruptor;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.std.*;
import org.jetbrains.annotations.NotNull;
......@@ -123,7 +123,7 @@ public class SampleByFillPrevRecordCursorFactory implements RecordCursorFactory
// empty map? this means that base cursor was empty
if (map.size() == 0) {
baseCursor.close();
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
}
// because we pass base cursor twice we have to go back to top
......
......@@ -31,7 +31,7 @@ import io.questdb.cairo.sql.RecordCursorFactory;
import io.questdb.cairo.sql.RecordMetadata;
import io.questdb.griffin.SqlException;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.griffin.model.ExpressionNode;
import io.questdb.std.Misc;
......@@ -90,7 +90,7 @@ public class SampleByFillValueNotKeyedRecordCursorFactory implements RecordCurso
return initFunctionsAndCursor(executionContext, baseCursor);
}
Misc.free(baseCursor);
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
}
@Override
......
......@@ -33,7 +33,7 @@ import io.questdb.cairo.sql.*;
import io.questdb.griffin.SqlException;
import io.questdb.griffin.SqlExecutionContext;
import io.questdb.griffin.SqlExecutionInterruptor;
import io.questdb.griffin.engine.EmptyTableRecordCursor;
import io.questdb.griffin.engine.EmptyTableNoSizeRecordCursor;
import io.questdb.griffin.engine.functions.GroupByFunction;
import io.questdb.griffin.engine.functions.constants.*;
import io.questdb.griffin.model.ExpressionNode;
......@@ -183,7 +183,7 @@ public class SampleByFillValueRecordCursorFactory implements RecordCursorFactory
// empty map? this means that base cursor was empty
if (map.size() == 0) {
baseCursor.close();
return EmptyTableRecordCursor.INSTANCE;
return EmptyTableNoSizeRecordCursor.INSTANCE;
}
// because we pass base cursor twice we have to go back to top
......
......@@ -125,9 +125,10 @@ public class AbstractGriffinTest extends AbstractCairoTest {
CharSequence expected,
RecordCursorFactory factory,
boolean supportsRandomAccess,
boolean checkSameStr
boolean checkSameStr,
boolean expectSize
) {
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, sqlExecutionContext);
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, expectSize, false, sqlExecutionContext);
}
protected static void assertCursor(
......@@ -135,6 +136,19 @@ public class AbstractGriffinTest extends AbstractCairoTest {
RecordCursorFactory factory,
boolean supportsRandomAccess,
boolean checkSameStr,
boolean expectSize,
boolean sizeCanBeVariable
) {
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable, sqlExecutionContext);
}
protected static void assertCursor(
CharSequence expected,
RecordCursorFactory factory,
boolean supportsRandomAccess,
boolean checkSameStr,
boolean sizeExpected,
boolean sizeCanBeVariable, // this means size() can either be -1 in some cases or known in others
SqlExecutionContext sqlExecutionContext
) {
try (RecordCursor cursor = factory.getCursor(sqlExecutionContext)) {
......@@ -169,6 +183,9 @@ public class AbstractGriffinTest extends AbstractCairoTest {
count++;
}
if (!sizeCanBeVariable) {
Assert.assertTrue((sizeExpected && cursorSize != -1) || (!sizeExpected && cursorSize == -1));
}
Assert.assertTrue(cursorSize == -1 || count == cursorSize);
TestUtils.assertEquals(expected, sink);
......@@ -362,14 +379,29 @@ public class AbstractGriffinTest extends AbstractCairoTest {
CharSequence ddl2,
CharSequence expected2,
boolean supportsRandomAccess,
boolean checkSameStr
boolean checkSameStr,
boolean expectSize
) throws SqlException {
printSqlResult(expected, query, expectedTimestamp, ddl2, expected2, supportsRandomAccess, checkSameStr, expectSize, false);
}
protected static void printSqlResult(
CharSequence expected,
CharSequence query,
CharSequence expectedTimestamp,
CharSequence ddl2,
CharSequence expected2,
boolean supportsRandomAccess,
boolean checkSameStr,
boolean expectSize,
boolean sizeCanBeVariable
) throws SqlException {
RecordCursorFactory factory = compiler.compile(query, sqlExecutionContext).getRecordCursorFactory();
try {
assertTimestamp(expectedTimestamp, factory);
assertCursor(expected, factory, supportsRandomAccess, checkSameStr);
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable);
// make sure we get the same outcome when we get factory to create new cursor
assertCursor(expected, factory, supportsRandomAccess, checkSameStr);
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable);
// make sure strings, binary fields and symbols are compliant with expected record behaviour
assertVariableColumns(factory, checkSameStr);
......@@ -379,9 +411,9 @@ public class AbstractGriffinTest extends AbstractCairoTest {
int count = 3;
while (count > 0) {
try {
assertCursor(expected2, factory, supportsRandomAccess, checkSameStr);
assertCursor(expected2, factory, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable);
// and again
assertCursor(expected2, factory, supportsRandomAccess, checkSameStr);
assertCursor(expected2, factory, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable);
return;
} catch (ReaderOutOfDateException e) {
Misc.free(factory);
......@@ -404,16 +436,18 @@ public class AbstractGriffinTest extends AbstractCairoTest {
@Nullable CharSequence ddl2,
@Nullable CharSequence expected2,
boolean supportsRandomAccess,
boolean checkSameStr
boolean checkSameStr,
boolean expectSize,
boolean sizeCanBeVariable
) throws Exception {
assertMemoryLeak(() -> {
if (ddl != null) {
compiler.compile(ddl, sqlExecutionContext);
}
if (verify != null) {
printSqlResult(null, verify, expectedTimestamp, ddl2, expected2, supportsRandomAccess, checkSameStr);
printSqlResult(null, verify, expectedTimestamp, ddl2, expected2, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable);
}
printSqlResult(expected, query, expectedTimestamp, ddl2, expected2, supportsRandomAccess, checkSameStr);
printSqlResult(expected, query, expectedTimestamp, ddl2, expected2, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable);
});
}
......@@ -423,7 +457,7 @@ public class AbstractGriffinTest extends AbstractCairoTest {
CharSequence ddl,
@Nullable CharSequence expectedTimestamp
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, true, true);
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, true, true, false, false);
}
protected static void assertQuery(
......@@ -433,7 +467,17 @@ public class AbstractGriffinTest extends AbstractCairoTest {
@Nullable CharSequence expectedTimestamp,
boolean supportsRandomAccess
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, supportsRandomAccess, true);
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, supportsRandomAccess, true, false, false);
}
protected static void assertQueryExpectSize(
CharSequence expected,
CharSequence query,
CharSequence ddl,
@Nullable CharSequence expectedTimestamp,
boolean supportsRandomAccess
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, supportsRandomAccess, true, true, false);
}
protected static void assertQuery(
......@@ -444,7 +488,19 @@ public class AbstractGriffinTest extends AbstractCairoTest {
boolean supportsRandomAccess,
boolean checkSameStr
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, supportsRandomAccess, checkSameStr);
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, supportsRandomAccess, checkSameStr, false, false);
}
protected static void assertQuery(
CharSequence expected,
CharSequence query,
CharSequence ddl,
@Nullable CharSequence expectedTimestamp,
boolean supportsRandomAccess,
boolean checkSameStr,
boolean expectSize
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, null, null, supportsRandomAccess, checkSameStr, expectSize, false);
}
protected static void assertQuery(
......@@ -453,8 +509,9 @@ public class AbstractGriffinTest extends AbstractCairoTest {
CharSequence ddl,
@Nullable CharSequence expectedTimestamp,
@Nullable CharSequence ddl2,
@Nullable CharSequence expected2) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, ddl2, expected2, true, true);
@Nullable CharSequence expected2
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, ddl2, expected2, true, true, false, false);
}
protected static void assertQuery(
......@@ -464,8 +521,38 @@ public class AbstractGriffinTest extends AbstractCairoTest {
@Nullable CharSequence expectedTimestamp,
@Nullable CharSequence ddl2,
@Nullable CharSequence expected2,
boolean supportsRandomAccess) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, ddl2, expected2, supportsRandomAccess, true);
boolean supportsRandomAccess
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, ddl2, expected2, supportsRandomAccess, true, false, false);
}
protected static void assertQuery(
CharSequence expected,
CharSequence query,
CharSequence ddl,
@Nullable CharSequence expectedTimestamp,
@Nullable CharSequence ddl2,
@Nullable CharSequence expected2,
boolean supportsRandomAccess,
boolean checkSameStr,
boolean expectSize
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, ddl2, expected2, supportsRandomAccess, checkSameStr, expectSize, false);
}
protected static void assertQuery(
CharSequence expected,
CharSequence query,
CharSequence ddl,
@Nullable CharSequence expectedTimestamp,
@Nullable CharSequence ddl2,
@Nullable CharSequence expected2,
boolean supportsRandomAccess,
boolean checkSameStr,
boolean expectSize,
boolean sizeCanBeVariable
) throws Exception {
assertQuery(expected, query, ddl, null, expectedTimestamp, ddl2, expected2, supportsRandomAccess, checkSameStr, expectSize, sizeCanBeVariable);
}
protected static void assertTimestamp(CharSequence expectedTimestamp, RecordCursorFactory factory) {
......@@ -502,16 +589,22 @@ public class AbstractGriffinTest extends AbstractCairoTest {
}
void assertFactoryCursor(String expected, String expectedTimestamp, RecordCursorFactory factory, boolean supportsRandomAccess, SqlExecutionContext sqlExecutionContext) {
assertFactoryCursor(expected, expectedTimestamp, factory, supportsRandomAccess, sqlExecutionContext, true);
assertFactoryCursor(expected, expectedTimestamp, factory, supportsRandomAccess, sqlExecutionContext, true, false);
}
void assertFactoryCursor(
String expected, String expectedTimestamp, RecordCursorFactory factory, boolean supportsRandomAccess, SqlExecutionContext sqlExecutionContext, boolean checkSameStr
String expected,
String expectedTimestamp,
RecordCursorFactory factory,
boolean supportsRandomAccess,
SqlExecutionContext sqlExecutionContext,
boolean checkSameStr,
boolean expectSize
) {
assertTimestamp(expectedTimestamp, factory, sqlExecutionContext);
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, sqlExecutionContext);
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, expectSize, false, sqlExecutionContext);
// make sure we get the same outcome when we get factory to create new cursor
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, sqlExecutionContext);
assertCursor(expected, factory, supportsRandomAccess, checkSameStr, expectSize, false, sqlExecutionContext);
// make sure strings, binary fields and symbols are compliant with expected record behaviour
assertVariableColumns(factory, checkSameStr);
}
......@@ -551,37 +644,82 @@ public class AbstractGriffinTest extends AbstractCairoTest {
assertQuery(compiler, expected, query, expectedTimestamp, supportsRandomAccess, sqlExecutionContext);
}
protected void assertQuery(String expected, String query, String expectedTimestamp, boolean supportsRandomAccess, boolean expectSize) throws SqlException {
assertQuery(compiler, expected, query, expectedTimestamp, sqlExecutionContext, supportsRandomAccess, true, expectSize);
}
protected void assertQuery(String expected, String query, String expectedTimestamp, boolean supportsRandomAccess, SqlExecutionContext sqlExecutionContext)
throws SqlException {
assertQuery(compiler, expected, query, expectedTimestamp, supportsRandomAccess, sqlExecutionContext, true);
assertQuery(compiler, expected, query, expectedTimestamp, sqlExecutionContext, supportsRandomAccess, true, false);
}
protected void assertQuery(SqlCompiler compiler, String expected, String query, String expectedTimestamp, boolean supportsRandomAccess, SqlExecutionContext sqlExecutionContext)
throws SqlException {
assertQuery(compiler, expected, query, expectedTimestamp, supportsRandomAccess, sqlExecutionContext, true);
assertQuery(compiler, expected, query, expectedTimestamp, sqlExecutionContext, supportsRandomAccess, true, false);
}
protected void assertQuery(
SqlCompiler compiler,
String expected,
String query,
String expectedTimestamp,
boolean supportsRandomAccess,
SqlExecutionContext sqlExecutionContext,
boolean expectSize
) throws SqlException {
assertQuery(compiler, expected, query, expectedTimestamp, sqlExecutionContext, supportsRandomAccess, true, expectSize);
}
protected void assertQuery(String expected, String query, String expectedTimestamp, boolean supportsRandomAccess, SqlExecutionContext sqlExecutionContext, boolean checkSameStr)
throws SqlException {
assertQuery(compiler, expected, query, expectedTimestamp, supportsRandomAccess, sqlExecutionContext, checkSameStr);
assertQuery(compiler, expected, query, expectedTimestamp, sqlExecutionContext, supportsRandomAccess, checkSameStr, false);
}
protected void assertQuery(
SqlCompiler compiler, String expected, String query, String expectedTimestamp, boolean supportsRandomAccess, SqlExecutionContext sqlExecutionContext, boolean checkSameStr
)
protected void assertQuery(String expected, String query, String expectedTimestamp, boolean supportsRandomAccess, SqlExecutionContext sqlExecutionContext, boolean checkSameStr, boolean expectSize)
throws SqlException {
assertQuery(compiler, expected, query, expectedTimestamp, sqlExecutionContext, supportsRandomAccess, checkSameStr, expectSize);
}
protected void assertQuery(
SqlCompiler compiler,
String expected,
String query,
String expectedTimestamp,
SqlExecutionContext sqlExecutionContext, boolean supportsRandomAccess,
boolean checkSameStr,
boolean expectSize
) throws SqlException {
try (final RecordCursorFactory factory = compiler.compile(query, sqlExecutionContext).getRecordCursorFactory()) {
assertFactoryCursor(expected, expectedTimestamp, factory, supportsRandomAccess, sqlExecutionContext, checkSameStr, expectSize);
}
}
protected void assertQueryAndCache(String expected, String query, String expectedTimestamp, boolean expectSize) throws SqlException {
assertQueryAndCache(expected, query, expectedTimestamp, false, expectSize);
}
protected void assertQueryAndCache(String expected, String query, String expectedTimestamp, boolean supportsRandomAccess, boolean expectSize) throws SqlException {
try (final RecordCursorFactory factory = compiler.compile(query, sqlExecutionContext).getRecordCursorFactory()) {
assertFactoryCursor(expected, expectedTimestamp, factory, supportsRandomAccess, sqlExecutionContext, checkSameStr);
assertFactoryCursor(expected, expectedTimestamp, factory, supportsRandomAccess, sqlExecutionContext, true, expectSize);
}
}
protected void assertQueryAndCache(String expected, String query, String expectedTimestamp) throws SqlException {
assertQueryAndCache(expected, query, expectedTimestamp, false);
protected void assertQueryPlain(
String expected,
String query
) throws SqlException {
try (final RecordCursorFactory factory = compiler.compile(query, sqlExecutionContext).getRecordCursorFactory()) {
assertFactoryCursor(expected, null, factory, true, sqlExecutionContext, true, true);
}
}
protected void assertQueryAndCache(String expected, String query, String expectedTimestamp, boolean supportsRandomAccess) throws SqlException {
protected void assertQueryPlain(
String expected,
String query,
@NotNull String timestamp
) throws SqlException {
try (final RecordCursorFactory factory = compiler.compile(query, sqlExecutionContext).getRecordCursorFactory()) {
assertFactoryCursor(expected, expectedTimestamp, factory, supportsRandomAccess);
assertFactoryCursor(expected, timestamp, factory, true, sqlExecutionContext, true, true);
}
}
}
......@@ -47,11 +47,6 @@ public class AlterTableAddColumnTest extends AbstractGriffinTest {
SharedRandom.RANDOM.set(new Rnd());
}
@Test
public void testAddBadSyntax() throws Exception {
assertFailure("alter table x add column abc int k", 33, "',' expected");
}
@Test
public void testAddBadColumnNameBackSlash() throws Exception {
assertFailure("alter table x add column \\", 25, "new column name contains invalid characters");
......@@ -67,6 +62,11 @@ public class AlterTableAddColumnTest extends AbstractGriffinTest {
assertFailure("alter table x add column /", 25, "new column name contains invalid characters");
}
@Test
public void testAddBadSyntax() throws Exception {
assertFailure("alter table x add column abc int k", 33, "',' expected");
}
@Test
public void testAddBusyTable() throws Exception {
assertMemoryLeak(() -> {
......@@ -120,7 +120,7 @@ public class AlterTableAddColumnTest extends AbstractGriffinTest {
Assert.assertEquals(ALTER, compiler.compile("alter table x add column mycol int", sqlExecutionContext).getType());
assertQuery(
assertQueryPlain(
"c\tmycol\n" +
"XYZ\tNaN\n" +
"ABC\tNaN\n" +
......@@ -132,9 +132,7 @@ public class AlterTableAddColumnTest extends AbstractGriffinTest {
"ABC\tNaN\n" +
"\tNaN\n" +
"XYZ\tNaN\n",
"select c, mycol from x",
null,
true
"select c, mycol from x"
);
}
);
......@@ -428,8 +426,7 @@ public class AlterTableAddColumnTest extends AbstractGriffinTest {
createX();
Assert.assertEquals(ALTER, compiler.compile("alter table x add column mycol int, second symbol", sqlExecutionContext).getType());
assertQuery(
assertQueryPlain(
"c\tmycol\tsecond\n" +
"XYZ\tNaN\t\n" +
"ABC\tNaN\t\n" +
......@@ -441,9 +438,7 @@ public class AlterTableAddColumnTest extends AbstractGriffinTest {
"ABC\tNaN\t\n" +
"\tNaN\t\n" +
"XYZ\tNaN\t\n",
"select c, mycol, second from x",
null,
true
"select c, mycol, second from x"
);
}
);
......
......@@ -44,21 +44,6 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
SharedRandom.RANDOM.set(new Rnd());
}
@Test
public void testBadSyntax() throws Exception {
assertFailure("alter table x alter column z", 28, "'add index' or 'cache' or 'nocache' expected");
}
@Test
public void testAlterFlagInNonSymbolColumn() throws Exception {
assertFailure("alter table x alter column b cache", 29, "Invalid column type - Column should be of type symbol");
}
@Test
public void testWhenCacheOrNocacheAreNotInAlterStatement() throws Exception {
assertFailure("alter table x alter column z ca", 29, "'cache' or 'nocache' expected");
}
@Test
public void testAlterExpectColumnKeyword() throws Exception {
assertFailure("alter table x alter", 19, "'column' expected");
......@@ -70,8 +55,8 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
}
@Test
public void testInvalidColumn() throws Exception {
assertFailure("alter table x alter column y cache", 29, "Invalid column: y");
public void testAlterFlagInNonSymbolColumn() throws Exception {
assertFailure("alter table x alter column b cache", 29, "Invalid column type - Column should be of type symbol");
}
@Test
......@@ -116,14 +101,14 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
final RecordCursorPrinter printer = new SingleColumnRecordCursorPrinter(sink, 1);
assertMemoryLeak(this::createX);
assertMemoryLeak(() -> {
assertQuery(expectedOrderedWhenCached,
"select sym from x order by sym",
"x",
null);
assertMemoryLeak(this::createX);
assertQueryPlain(expectedOrderedWhenCached,
"select sym from x order by sym"
);
assertMemoryLeak(() -> {
try (TableReader reader = engine.getReader(sqlExecutionContext.getCairoSecurityContext(), "x")) {
//check cursor before altering symbol column
sink.clear();
......@@ -148,10 +133,9 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
}
});
assertQuery(expectedUnordered,
"select sym from x order by 1 asc",
"x",
null);
assertQueryPlain(expectedUnordered,
"select sym from x order by 1 asc"
);
}
@Test
......@@ -171,7 +155,6 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
executeInsert("insert into x values (9, 'GBP')\"");
});
String expectUnordered = "sym\n" +
"GBP\n" +
"GBP\n" +
......@@ -182,10 +165,6 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
"GBP\n" +
"CHF\n" +
"GBP\n";
assertQuery(expectUnordered,
"select sym from x order by sym",
"x",
null);
String expected = "sym\n" +
"GBP\n" +
......@@ -199,6 +178,11 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
"GBP\n";
assertMemoryLeak(() -> {
assertQueryPlain(expectUnordered,
"select sym from x order by sym"
);
try (TableReader reader = engine.getReader(sqlExecutionContext.getCairoSecurityContext(), "x")) {
//check cursor before altering symbol column
sink.clear();
......@@ -234,10 +218,25 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
"GBP\n" +
"JPY\n" +
"USD\n";
assertQuery(expectedOrdered,
"select sym from x order by 1 asc",
"x",
null);
assertQueryPlain(expectedOrdered,
"select sym from x order by 1 asc"
);
}
@Test
public void testBadSyntax() throws Exception {
assertFailure("alter table x alter column z", 28, "'add index' or 'cache' or 'nocache' expected");
}
@Test
public void testInvalidColumn() throws Exception {
assertFailure("alter table x alter column y cache", 29, "Invalid column: y");
}
@Test
public void testWhenCacheOrNocacheAreNotInAlterStatement() throws Exception {
assertFailure("alter table x alter column z ca", 29, "'cache' or 'nocache' expected");
}
private void assertFailure(String sql, int position, String message) throws Exception {
......@@ -288,12 +287,6 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
this.columnIndex = columnIndex;
}
@Override
public void printHeader(RecordMetadata metadata) {
sink.put(metadata.getColumnName(columnIndex));
sink.put('\n');
}
@Override
public void print(Record r, RecordMetadata m) {
printColumn(r, m, columnIndex);
......@@ -301,5 +294,11 @@ public class AlterTableAlterSymbolColumnCacheFlagTest extends AbstractGriffinTes
sink.flush();
}
@Override
public void printHeader(RecordMetadata metadata) {
sink.put(metadata.getColumnName(columnIndex));
sink.put('\n');
}
}
}
......@@ -284,6 +284,8 @@ public class InsertTest extends AbstractGriffinTest {
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(10)) timestamp (ts)",
expected,
true,
true,
true
);
}
......@@ -470,6 +472,8 @@ public class InsertTest extends AbstractGriffinTest {
"ts",
"insert into tab select x ac, timestamp_sequence(0, x) ts from long_sequence(10)",
expected,
true,
true,
true
);
}
......
......@@ -38,20 +38,6 @@ public class KeyedAggregationTest extends AbstractGriffinTest {
SharedRandom.RANDOM.set(new Rnd());
}
@Test
public void testIntSymbolResolution() throws Exception {
assertQuery(
"s2\tsum\n" +
"\t104119.880948161\n" +
"a1\t103804.62242300605\n" +
"a2\t104433.68659571148\n" +
"a3\t104341.28852517322\n",
"select s2, sum(val) from tab order by s2",
"create table tab as (select rnd_symbol('s1','s2','s3', null) s1, rnd_symbol('a1','a2','a3', null) s2, rnd_double(2) val from long_sequence(1000000))",
null, true
);
}
@Test
public void testHourDouble() throws Exception {
assertQuery(
......@@ -61,46 +47,46 @@ public class KeyedAggregationTest extends AbstractGriffinTest {
"2\t11641.765471468498\t11641.76547146845\t11641.765471468458\t1.8566421983501336E-5\t0.9999768905891359\t0.500376750256533\t1970-01-01T02:46:39.900000Z\t1970-01-01T02:00:00.000000Z\n",
"select hour(ts), sum(val), ksum(val), nsum(val), min(val), max(val), avg(val), max(ts), min(ts) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_double(2) val from long_sequence(100000))",
null, true
null, true, true, true
);
}
@Test
public void testHourLong() throws Exception {
public void testHourFiltered() throws Exception {
assertQuery(
"hour\tcount\tsum\tmin\tmax\tavg\n" +
"0\t36000\t13265789485\t-988\t889951\t443212.3712872941\n" +
"1\t36000\t13359838134\t-997\t889948\t444350.36699261627\n" +
"2\t28000\t10444993989\t-992\t889982\t445586.53594129946\n",
"select hour(ts), count(), sum(val), min(val), max(val), avg(val) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_long(-998, 889991, 2) val from long_sequence(100000))",
null, true
"hour\tcount\n" +
"0\t36000\n" +
"1\t36000\n" +
"2\t28000\n",
"select hour(ts), count() from tab where val < 0.5",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_double() val from long_sequence(100000))",
null, true, true, true
);
}
@Test
public void testHourPossibleBugfix() throws Exception {
public void testHourInt() throws Exception {
assertQuery(
"hour\tsum\tksum\tnsum\tmin\tmax\tavg\n" +
"0\t13265789485\t1.3265789485E10\t1.3265789485E10\t-988\t889951\t443212.3712872941\n" +
"1\t13359838134\t1.3359838134E10\t1.3359838134E10\t-997\t889948\t444350.36699261627\n" +
"2\t10444993989\t1.0444993989E10\t1.0444993989E10\t-992\t889982\t445586.53594129946\n",
"select hour(ts), sum(val), ksum(val), nsum(val), min(val), max(val), avg(val) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_long(-998, 889991, 2) val from long_sequence(100000))",
null, true
"hour\tcount\tsum\tmin\tmax\tavg\n" +
"0\t36000\t13332495967\t-995\t889975\t445441.0466406067\n" +
"1\t36000\t13360114022\t-950\t889928\t444359.54307190847\n" +
"2\t28000\t10420189893\t-914\t889980\t444528.3858623779\n",
"select hour(ts), count(), sum(val), min(val), max(val), avg(val) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_int(-998, 889991, 2) val from long_sequence(100000))",
null, true, true, true
);
}
@Test
public void testHourFiltered() throws Exception {
public void testHourLong() throws Exception {
assertQuery(
"hour\tcount\n" +
"0\t36000\n" +
"1\t36000\n" +
"2\t28000\n",
"select hour(ts), count() from tab where val < 0.5",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_double() val from long_sequence(100000))",
null, true
"hour\tcount\tsum\tmin\tmax\tavg\n" +
"0\t36000\t13265789485\t-988\t889951\t443212.3712872941\n" +
"1\t36000\t13359838134\t-997\t889948\t444350.36699261627\n" +
"2\t28000\t10444993989\t-992\t889982\t445586.53594129946\n",
"select hour(ts), count(), sum(val), min(val), max(val), avg(val) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_long(-998, 889991, 2) val from long_sequence(100000))",
null, true, true, true
);
}
......@@ -113,20 +99,34 @@ public class KeyedAggregationTest extends AbstractGriffinTest {
"2\t1.0444993989E10\t1.0444993989E10\n",
"select hour(ts), ksum(val), nsum(val) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_long(-998, 889991, 2) val from long_sequence(100000))",
null, true
null, true, true, true
);
}
@Test
public void testHourInt() throws Exception {
public void testHourPossibleBugfix() throws Exception {
assertQuery(
"hour\tcount\tsum\tmin\tmax\tavg\n" +
"0\t36000\t13332495967\t-995\t889975\t445441.0466406067\n" +
"1\t36000\t13360114022\t-950\t889928\t444359.54307190847\n" +
"2\t28000\t10420189893\t-914\t889980\t444528.3858623779\n",
"select hour(ts), count(), sum(val), min(val), max(val), avg(val) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_int(-998, 889991, 2) val from long_sequence(100000))",
null, true
"hour\tsum\tksum\tnsum\tmin\tmax\tavg\n" +
"0\t13265789485\t1.3265789485E10\t1.3265789485E10\t-988\t889951\t443212.3712872941\n" +
"1\t13359838134\t1.3359838134E10\t1.3359838134E10\t-997\t889948\t444350.36699261627\n" +
"2\t10444993989\t1.0444993989E10\t1.0444993989E10\t-992\t889982\t445586.53594129946\n",
"select hour(ts), sum(val), ksum(val), nsum(val), min(val), max(val), avg(val) from tab order by 1",
"create table tab as (select timestamp_sequence(0, 100000) ts, rnd_long(-998, 889991, 2) val from long_sequence(100000))",
null, true, true, true
);
}
@Test
public void testIntSymbolResolution() throws Exception {
assertQuery(
"s2\tsum\n" +
"\t104119.880948161\n" +
"a1\t103804.62242300605\n" +
"a2\t104433.68659571148\n" +
"a3\t104341.28852517322\n",
"select s2, sum(val) from tab order by s2",
"create table tab as (select rnd_symbol('s1','s2','s3', null) s1, rnd_symbol('a1','a2','a3', null) s2, rnd_double(2) val from long_sequence(1000000))",
null, true, true, true
);
}
......
......@@ -263,11 +263,11 @@ public class LimitTest extends AbstractGriffinTest {
bindVariableService.setLong("lo", 4);
bindVariableService.setInt("hi", 8);
assertQueryAndCache(expected1, query, "timestamp", true);
assertQueryAndCache(expected1, query, "timestamp", true, true);
bindVariableService.setLong("lo", 6);
bindVariableService.setInt("hi", 12);
assertQueryAndCache(expected2, query, "timestamp", true);
assertQueryAndCache(expected2, query, "timestamp", true, true);
} finally {
engine.releaseAllWriters();
engine.releaseAllReaders();
......@@ -449,9 +449,9 @@ public class LimitTest extends AbstractGriffinTest {
);
bindVariableService.setLong(0, 4);
assertQueryAndCache(expected1, query, "timestamp", true);
assertQueryAndCache(expected1, query, "timestamp", true, true);
bindVariableService.setLong(0, 6);
assertQueryAndCache(expected2, query, "timestamp", true);
assertQueryAndCache(expected2, query, "timestamp", true, true);
} finally {
engine.releaseAllWriters();
engine.releaseAllReaders();
......@@ -505,9 +505,9 @@ public class LimitTest extends AbstractGriffinTest {
);
bindVariableService.setLong("lim", 4);
assertQueryAndCache(expected1, query, "timestamp", true);
assertQueryAndCache(expected1, query, "timestamp", true, true);
bindVariableService.setLong("lim", 6);
assertQueryAndCache(expected2, query, "timestamp", true);
assertQueryAndCache(expected2, query, "timestamp", true, true);
} finally {
engine.releaseAllWriters();
engine.releaseAllReaders();
......@@ -550,7 +550,7 @@ public class LimitTest extends AbstractGriffinTest {
") timestamp(timestamp)"
, sqlExecutionContext
);
assertQueryAndCache(expected1, query, "timestamp", true);
assertQueryAndCache(expected1, query, "timestamp", true, true);
compiler.compile(
"insert into y select * from " +
......@@ -576,7 +576,7 @@ public class LimitTest extends AbstractGriffinTest {
, sqlExecutionContext
);
assertQuery(expected2, query, "timestamp", true);
assertQuery(expected2, query, "timestamp", true, true);
});
}
}
......@@ -59,35 +59,38 @@ public class NoopGroupByTest extends AbstractGriffinTest {
"hour\tavgBid\n" +
"0\t0.47607185409853914\n" +
"1\t0.6861237948732989\n",
true,
true,
true
);
}
@Test
public void testSubQuery() throws Exception {
public void testNoopGroupByValidColumnName() throws Exception {
assertQuery(
"bkt\tavg\n",
"select bkt, avg(bid) from (select abs(id % 10) bkt, bid from x) group by bkt",
"sym1\tavgBid\n",
"select a.sym1, avg(bid) avgBid from x a group by a.sym1 order by a.sym1",
"create table x (\n" +
" id long,\n" +
" bid double\n" +
") ",
" sym1 symbol,\n" +
" sym2 symbol,\n" +
" bid double,\n" +
" ask double,\n" +
" ts timestamp\n" +
") timestamp(ts) partition by DAY",
null,
"insert into x select * from (select " +
" rnd_long(), \n" +
" rnd_double() \n" +
" from long_sequence(20))",
"bkt\tavg\n" +
"6\t0.7275909062911847\n" +
"5\t0.08486964232560668\n" +
"8\t0.5773046624150107\n" +
"4\t0.413662826357355\n" +
"2\t0.22452340856088226\n" +
"1\t0.33762525947485594\n" +
"3\t0.7715455271652294\n" +
"7\t0.47335449523280454\n" +
"0\t0.1911234617573182\n" +
"9\t0.5793466326862211\n",
" rnd_symbol('A', 'B', 'C') sym1, \n" +
" rnd_symbol('D', 'E', 'F') sym2, \n" +
" rnd_double() bid, \n" +
" rnd_double() ask, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(20)) timestamp (ts)",
"sym1\tavgBid\n" +
"A\t0.5942181417903911\n" +
"B\t0.7080299543021055\n" +
"C\t0.4760584891454253\n",
true,
true,
true
);
}
......@@ -229,10 +232,11 @@ public class NoopGroupByTest extends AbstractGriffinTest {
}
@Test
public void testNoopGroupByValidColumnName() throws Exception {
public void testNoopGroupByValidColumnNameWithHourFunction() throws Exception {
assertQuery(
"sym1\tavgBid\n",
"select a.sym1, avg(bid) avgBid from x a group by a.sym1 order by a.sym1",
"hour\tavgBid\n",
//select hour(pickup_datetime), sum(passenger_count) from trips group by hour(pickup_datetime);
"select hour(ts), avg(bid) avgBid from x group by hour(ts) order by hour",
"create table x (\n" +
" sym1 symbol,\n" +
" sym2 symbol,\n" +
......@@ -248,20 +252,21 @@ public class NoopGroupByTest extends AbstractGriffinTest {
" rnd_double() ask, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(20)) timestamp (ts)",
"sym1\tavgBid\n" +
"A\t0.5942181417903911\n" +
"B\t0.7080299543021055\n" +
"C\t0.4760584891454253\n",
"hour\tavgBid\n" +
"0\t0.47607185409853914\n" +
"1\t0.6861237948732989\n",
true,
true,
true
);
}
@Test
public void testNoopGroupByValidColumnNameWithHourFunction() throws Exception {
public void testNoopGroupByValidColumnNameWithHourFunctionAndAliasedTable() throws Exception {
assertQuery(
"hour\tavgBid\n",
//select hour(pickup_datetime), sum(passenger_count) from trips group by hour(pickup_datetime);
"select hour(ts), avg(bid) avgBid from x group by hour(ts) order by hour",
"select hour(a.ts), avg(bid) avgBid from x a group by hour(a.ts) order by hour",
"create table x (\n" +
" sym1 symbol,\n" +
" sym2 symbol,\n" +
......@@ -280,16 +285,17 @@ public class NoopGroupByTest extends AbstractGriffinTest {
"hour\tavgBid\n" +
"0\t0.47607185409853914\n" +
"1\t0.6861237948732989\n",
true,
true,
true
);
}
@Test
public void testNoopGroupByValidColumnNameWithHourFunctionAndAliasedTable() throws Exception {
public void testNoopGroupByWhenUsingAliasedColumn() throws Exception {
assertQuery(
"hour\tavgBid\n",
//select hour(pickup_datetime), sum(passenger_count) from trips group by hour(pickup_datetime);
"select hour(a.ts), avg(bid) avgBid from x a group by hour(a.ts) order by hour",
"ccy\tavgBid\n",
"select sym1 ccy, avg(bid) avgBid from x where sym1 in ('A', 'B' ) group by ccy",
"create table x (\n" +
" sym1 symbol,\n" +
" sym2 symbol,\n" +
......@@ -305,18 +311,20 @@ public class NoopGroupByTest extends AbstractGriffinTest {
" rnd_double() ask, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(20)) timestamp (ts)",
"hour\tavgBid\n" +
"0\t0.47607185409853914\n" +
"1\t0.6861237948732989\n",
"ccy\tavgBid\n" +
"A\t0.5942181417903911\n" +
"B\t0.7080299543021055\n",
true,
true,
true
);
}
@Test
public void testNoopGroupByWhenUsingAliasedColumn() throws Exception {
public void testNoopGroupByWhenUsingAliasedColumnAndAliasedTable() throws Exception {
assertQuery(
"ccy\tavgBid\n",
"select sym1 ccy, avg(bid) avgBid from x where sym1 in ('A', 'B' ) group by ccy",
"select sym1 ccy, avg(bid) avgBid from x a where sym1 in ('A', 'B' ) group by a.ccy",
"create table x (\n" +
" sym1 symbol,\n" +
" sym2 symbol,\n" +
......@@ -335,15 +343,17 @@ public class NoopGroupByTest extends AbstractGriffinTest {
"ccy\tavgBid\n" +
"A\t0.5942181417903911\n" +
"B\t0.7080299543021055\n",
true,
true,
true
);
}
@Test
public void testNoopGroupByWhenUsingAliasedColumnAndAliasedTable() throws Exception {
public void testNoopGroupByWith1Syms() throws Exception {
assertQuery(
"ccy\tavgBid\n",
"select sym1 ccy, avg(bid) avgBid from x a where sym1 in ('A', 'B' ) group by a.ccy",
"sym1\tavgBid\n",
"select sym1, avg(bid) avgBid from x where sym1 in ('A', 'B' ) group by sym1",
"create table x (\n" +
" sym1 symbol,\n" +
" sym2 symbol,\n" +
......@@ -359,9 +369,11 @@ public class NoopGroupByTest extends AbstractGriffinTest {
" rnd_double() ask, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(20)) timestamp (ts)",
"ccy\tavgBid\n" +
"sym1\tavgBid\n" +
"A\t0.5942181417903911\n" +
"B\t0.7080299543021055\n",
true,
true,
true
);
}
......@@ -383,10 +395,10 @@ public class NoopGroupByTest extends AbstractGriffinTest {
}
@Test
public void testNoopGroupByWith1Syms() throws Exception {
public void testNoopGroupByWith2Syms() throws Exception {
assertQuery(
"sym1\tavgBid\n",
"select sym1, avg(bid) avgBid from x where sym1 in ('A', 'B' ) group by sym1",
"sym1\tsym2\tavgBid\n",
"select sym1, sym2, avg(bid) avgBid from x where sym1 in ('A', 'B' ) group by sym1, sym2",
"create table x (\n" +
" sym1 symbol,\n" +
" sym2 symbol,\n" +
......@@ -402,9 +414,14 @@ public class NoopGroupByTest extends AbstractGriffinTest {
" rnd_double() ask, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(20)) timestamp (ts)",
"sym1\tavgBid\n" +
"A\t0.5942181417903911\n" +
"B\t0.7080299543021055\n",
"sym1\tsym2\tavgBid\n" +
"A\tD\t0.47381585528154324\n" +
"B\tE\t0.6403134139386097\n" +
"A\tE\t0.5837537495691357\n" +
"B\tD\t0.8434630350290969\n" +
"A\tF\t0.8664158914718532\n",
true,
true,
true
);
}
......@@ -432,36 +449,8 @@ public class NoopGroupByTest extends AbstractGriffinTest {
"sym1\tavgBid\n" +
"A\t0.5942181417903911\n" +
"B\t0.7080299543021055\n",
true
);
}
@Test
public void testNoopGroupByWith2Syms() throws Exception {
assertQuery(
"sym1\tsym2\tavgBid\n",
"select sym1, sym2, avg(bid) avgBid from x where sym1 in ('A', 'B' ) group by sym1, sym2",
"create table x (\n" +
" sym1 symbol,\n" +
" sym2 symbol,\n" +
" bid double,\n" +
" ask double,\n" +
" ts timestamp\n" +
") timestamp(ts) partition by DAY",
null,
"insert into x select * from (select " +
" rnd_symbol('A', 'B', 'C') sym1, \n" +
" rnd_symbol('D', 'E', 'F') sym2, \n" +
" rnd_double() bid, \n" +
" rnd_double() ask, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(20)) timestamp (ts)",
"sym1\tsym2\tavgBid\n" +
"A\tD\t0.47381585528154324\n" +
"B\tE\t0.6403134139386097\n" +
"A\tE\t0.5837537495691357\n" +
"B\tD\t0.8434630350290969\n" +
"A\tF\t0.8664158914718532\n",
true,
true,
true
);
}
......@@ -509,6 +498,39 @@ public class NoopGroupByTest extends AbstractGriffinTest {
"0.9144934765891063\t0.6551335839796312\n" +
"0.7675889012481835\t0.9540069089049732\n" +
"0.9257619753148886\t0.19751370382305056\n",
true,
true,
true
);
}
@Test
public void testSubQuery() throws Exception {
assertQuery(
"bkt\tavg\n",
"select bkt, avg(bid) from (select abs(id % 10) bkt, bid from x) group by bkt",
"create table x (\n" +
" id long,\n" +
" bid double\n" +
") ",
null,
"insert into x select * from (select " +
" rnd_long(), \n" +
" rnd_double() \n" +
" from long_sequence(20))",
"bkt\tavg\n" +
"6\t0.7275909062911847\n" +
"5\t0.08486964232560668\n" +
"8\t0.5773046624150107\n" +
"4\t0.413662826357355\n" +
"2\t0.22452340856088226\n" +
"1\t0.33762525947485594\n" +
"3\t0.7715455271652294\n" +
"7\t0.47335449523280454\n" +
"0\t0.1911234617573182\n" +
"9\t0.5793466326862211\n",
true,
true,
true
);
}
......
......@@ -35,6 +35,8 @@ public class QuoteIdentifierTest extends AbstractGriffinTest {
"quoted_table",
"create table \"quoted_table\"(\"id\" long,\"name\" string)",
null,
true,
true,
true
);
}
......
......@@ -72,6 +72,8 @@ public class RenameTableTest extends AbstractGriffinTest {
null,
"timestamp",
true,
true,
true
);
}
......
......@@ -154,32 +154,25 @@ public class SecurityTest extends AbstractGriffinTest {
}
@Test
public void testInsertDeniedOnNoWriteAccess() throws Exception {
public void testAlterTableDeniedOnNoWriteAccess() throws Exception {
assertMemoryLeak(() -> {
compiler.compile("create table balances(cust_id int, ccy symbol, balance double)", sqlExecutionContext);
assertQuery("count\n0\n", "select count() from balances", null);
CompiledQuery cq = compiler.compile("insert into balances values (1, 'EUR', 140.6)", sqlExecutionContext);
InsertStatement insertStatement = cq.getInsertStatement();
try (InsertMethod method = insertStatement.createMethod(sqlExecutionContext)) {
method.execute();
method.commit();
}
assertQuery("count\n1\n", "select count() from balances", null);
assertQuery("cust_id\tccy\tbalance\n1\tEUR\t140.6\n", "select * from balances", null, true, true);
try {
cq = compiler.compile("insert into balances values (2, 'ZAR', 140.6)", readOnlyExecutionContext);
insertStatement = cq.getInsertStatement();
try (InsertMethod method = insertStatement.createMethod(readOnlyExecutionContext)) {
method.execute();
method.commit();
}
compiler.compile("alter table balances add column newcol int", readOnlyExecutionContext);
Assert.fail();
} catch (Exception ex) {
Assert.assertTrue(ex.toString().contains("permission denied"));
}
assertQuery("count\n1\n", "select count() from balances", null);
assertQueryPlain("cust_id\tccy\tbalance\n1\tEUR\t140.6\n", "select * from balances");
});
}
......@@ -193,44 +186,75 @@ public class SecurityTest extends AbstractGriffinTest {
} catch (Exception ex) {
Assert.assertTrue(ex.toString().contains("permission denied"));
}
assertQuery("count\n0\n", "select count() from balances", null);
assertQuery("count\n0\n", "select count() from balances", null, false, true);
});
}
@Test
public void testAlterTableDeniedOnNoWriteAccess() throws Exception {
public void testInsertDeniedOnNoWriteAccess() throws Exception {
assertMemoryLeak(() -> {
compiler.compile("create table balances(cust_id int, ccy symbol, balance double)", sqlExecutionContext);
assertQuery("count\n0\n", "select count() from balances", null, false, true);
CompiledQuery cq = compiler.compile("insert into balances values (1, 'EUR', 140.6)", sqlExecutionContext);
InsertStatement insertStatement = cq.getInsertStatement();
try (InsertMethod method = insertStatement.createMethod(sqlExecutionContext)) {
method.execute();
method.commit();
}
assertQuery("cust_id\tccy\tbalance\n1\tEUR\t140.6\n", "select * from balances", null, true);
assertQuery("count\n1\n", "select count() from balances", null, false, true);
try {
compiler.compile("alter table balances add column newcol int", readOnlyExecutionContext);
cq = compiler.compile("insert into balances values (2, 'ZAR', 140.6)", readOnlyExecutionContext);
insertStatement = cq.getInsertStatement();
try (InsertMethod method = insertStatement.createMethod(readOnlyExecutionContext)) {
method.execute();
method.commit();
}
Assert.fail();
} catch (Exception ex) {
Assert.assertTrue(ex.toString().contains("permission denied"));
}
assertQuery("cust_id\tccy\tbalance\n1\tEUR\t140.6\n", "select * from balances", null, true);
assertQuery("count\n1\n", "select count() from balances", null, false, true);
});
}
@Test
public void testRenameTableDeniedOnNoWriteAccess() throws Exception {
public void testInterruptorWithNonKeyedAgg() throws Exception {
assertMemoryLeak(() -> {
compiler.compile("create table balances(cust_id int, ccy symbol, balance double)", sqlExecutionContext);
sqlExecutionContext.getRandom().reset();
compiler.compile("create table tb1 as (select" +
" rnd_symbol(3,3,3,20000) sym1," +
" rnd_double(2) d1," +
" timestamp_sequence(0, 1000000000) ts1" +
" from long_sequence(10000)) timestamp(ts1)", sqlExecutionContext);
assertQuery(
memoryRestrictedCompiler,
"sum\n" +
"165.6121723103405\n",
"select sum(d1) from tb1 where d1 < 0.2",
null,
false,
readOnlyExecutionContext,
true
);
Assert.assertTrue(nCheckInterruptedCalls.get() > 0);
try {
compiler.compile("rename table balances to newname", readOnlyExecutionContext);
setMaxInterruptorChecks(2);
assertQuery(
memoryRestrictedCompiler,
"sym1\nWCP\nICC\nUOJ\nFJG\nOZZ\nGHV\nWEK\nVDZ\nETJ\nUED\n",
"select sum(d1) from tb1 where d1 < 0.2",
null,
true,
readOnlyExecutionContext,
true
);
Assert.fail();
} catch (Exception ex) {
Assert.assertTrue(ex.toString().contains("permission denied"));
Assert.assertTrue(ex.toString().contains("Interrupting SQL processing, max calls is 2"));
}
assertQuery("count\n0\n", "select count() from balances", null);
});
}
......@@ -517,34 +541,16 @@ public class SecurityTest extends AbstractGriffinTest {
}
@Test
public void testTreeResizesWithImplicitGroupBy() throws Exception {
SqlExecutionContext readOnlyExecutionContext = new SqlExecutionContextImpl(engine, 1)
.with(new CairoSecurityContextImpl(false), bindVariableService, null, -1, null);
public void testRenameTableDeniedOnNoWriteAccess() throws Exception {
assertMemoryLeak(() -> {
sqlExecutionContext.getRandom().reset();
compiler.compile("create table tb1 as (select" +
" rnd_symbol(4,4,4,20000) sym1," +
" rnd_symbol(2,2,2,20000) sym2," +
" rnd_double(2) d," +
" timestamp_sequence(0, 1000000000) ts" +
" from long_sequence(2000)) timestamp(ts)", sqlExecutionContext);
assertQuery(
memoryRestrictedCompiler,
"sym2\tcount\nGZ\t1040\nRX\t960\n",
"select sym2, count() from tb1 order by sym2",
null,
true, readOnlyExecutionContext);
compiler.compile("create table balances(cust_id int, ccy symbol, balance double)", sqlExecutionContext);
try {
assertQuery(
memoryRestrictedCompiler,
"sym1\tcount\nPEHN\t265\nCPSW\t231\nHYRX\t262\nVTJW\t242\n",
"select sym1, count() from tb1 order by sym1",
null,
true, readOnlyExecutionContext);
compiler.compile("rename table balances to newname", readOnlyExecutionContext);
Assert.fail();
} catch (Exception ex) {
Assert.assertTrue(ex.toString().contains("Maximum number of pages (2) breached"));
Assert.assertTrue(ex.toString().contains("permission denied"));
}
assertQuery("count\n0\n", "select count() from balances", null, false, true);
});
}
......@@ -619,37 +625,39 @@ public class SecurityTest extends AbstractGriffinTest {
}
@Test
public void testInterruptorWithNonKeyedAgg() throws Exception {
public void testTreeResizesWithImplicitGroupBy() throws Exception {
SqlExecutionContext readOnlyExecutionContext = new SqlExecutionContextImpl(engine, 1)
.with(new CairoSecurityContextImpl(false), bindVariableService, null, -1, null);
assertMemoryLeak(() -> {
sqlExecutionContext.getRandom().reset();
compiler.compile("create table tb1 as (select" +
" rnd_symbol(3,3,3,20000) sym1," +
" rnd_double(2) d1," +
" timestamp_sequence(0, 1000000000) ts1" +
" from long_sequence(10000)) timestamp(ts1)", sqlExecutionContext);
" rnd_symbol(4,4,4,20000) sym1," +
" rnd_symbol(2,2,2,20000) sym2," +
" rnd_double(2) d," +
" timestamp_sequence(0, 1000000000) ts" +
" from long_sequence(2000)) timestamp(ts)", sqlExecutionContext);
assertQuery(
memoryRestrictedCompiler,
"sum\n" +
"165.6121723103405\n",
"select sum(d1) from tb1 where d1 < 0.2",
"sym2\tcount\nGZ\t1040\nRX\t960\n",
"select sym2, count() from tb1 order by sym2",
null,
false,
readOnlyExecutionContext
true,
readOnlyExecutionContext,
true
);
Assert.assertTrue(nCheckInterruptedCalls.get() > 0);
try {
setMaxInterruptorChecks(2);
assertQuery(
memoryRestrictedCompiler,
"sym1\nWCP\nICC\nUOJ\nFJG\nOZZ\nGHV\nWEK\nVDZ\nETJ\nUED\n",
"select sum(d1) from tb1 where d1 < 0.2",
"sym1\tcount\nPEHN\t265\nCPSW\t231\nHYRX\t262\nVTJW\t242\n",
"select sym1, count() from tb1 order by sym1",
null,
readOnlyExecutionContext, true,
true,
readOnlyExecutionContext
true
);
Assert.fail();
} catch (Exception ex) {
Assert.assertTrue(ex.toString().contains("Interrupting SQL processing, max calls is 2"));
Assert.assertTrue(ex.toString().contains("Maximum number of pages (2) breached"));
}
});
}
......
......@@ -69,52 +69,48 @@ public class SymbolNotEqualsValueTest extends AbstractGriffinTest {
}
@Test
public void testNotEquals1SymbolsWithInvariantOrderBy() throws Exception {
final String expected = "k\tprice\tts\n" +
"DXR\t0.0843832076262595\t1970-01-03T00:12:00.000000Z\n" +
"DXR\t0.08486964232560668\t1970-01-03T00:06:00.000000Z\n" +
"HBC\t0.0367581207471136\t1970-01-03T00:54:00.000000Z\n" +
"HBC\t0.7905675319675964\t1970-01-03T00:24:00.000000Z\n" +
"HBC\t0.6508594025855301\t1970-01-03T00:18:00.000000Z\n";
public void testNotEquals1SymbolsWithConstantFilter() throws Exception {
final String expected = "k\tj\tprice\tts\n";
assertQuery(
"k\tprice\tts\n",
"select sym k, price, ts from x where sym != 'ABB' order by k",
"k\tj\tprice\tts\n",
"select sym k, sym2 j, price, ts from x where sym != 'ABB' and 2 = 1",
"create table x (\n" +
" sym symbol cache index,\n" +
" sym2 symbol cache index,\n" +
" price double,\n" +
" ts timestamp\n" +
") timestamp(ts) partition by DAY",
null,
"ts",
"insert into x select * from (select rnd_symbol('ABB', 'HBC', 'DXR') sym, \n" +
" rnd_symbol('D', 'E', 'F') sym2, \n" +
" rnd_double() price, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(10)) timestamp (ts)",
" from long_sequence(10)) timestamp (ts)",
expected,
false,
true,
true
);
}
@Test
public void testNotEquals1SymbolsWithInvariantOrderBy2SymsDesc() throws Exception {
public void testNotEquals1SymbolsWithFilter() throws Exception {
final String expected = "k\tj\tprice\tts\n" +
"DXR\tF\t0.6778564558839208\t1970-01-03T00:48:00.000000Z\n" +
"DXR\tF\t0.299199045961845\t1970-01-03T00:06:00.000000Z\n" +
"DXR\tD\t0.38539947865244994\t1970-01-03T00:54:00.000000Z\n" +
"HBC\tE\t0.9856290845874263\t1970-01-03T00:18:00.000000Z\n" +
"HBC\tD\t0.2390529010846525\t1970-01-03T00:42:00.000000Z\n" +
"HBC\tD\t0.7611029514995744\t1970-01-03T00:30:00.000000Z\n";
"HBC\tD\t0.7611029514995744\t1970-01-03T00:30:00.000000Z\n" +
"DXR\tF\t0.6778564558839208\t1970-01-03T00:48:00.000000Z\n";
assertQuery(
"k\tj\tprice\tts\n",
"select sym k, sym2 j, price, ts from x where sym != 'ABB' order by k, j desc",
"select sym k, sym2 j, price, ts from x where sym != 'ABB' and price > 0.5",
"create table x (\n" +
" sym symbol cache index,\n" +
" sym2 symbol cache index,\n" +
" price double,\n" +
" ts timestamp\n" +
") timestamp(ts) partition by DAY",
null,
"ts",
"insert into x select * from (select rnd_symbol('ABB', 'HBC', 'DXR') sym, \n" +
" rnd_symbol('D', 'E', 'F') sym2, \n" +
" rnd_double() price, \n" +
......@@ -126,46 +122,52 @@ public class SymbolNotEqualsValueTest extends AbstractGriffinTest {
}
@Test
public void testNotEquals1SymbolsWithConstantFilter() throws Exception {
final String expected = "k\tj\tprice\tts\n";
public void testNotEquals1SymbolsWithInvariantOrderBy() throws Exception {
final String expected = "k\tprice\tts\n" +
"DXR\t0.0843832076262595\t1970-01-03T00:12:00.000000Z\n" +
"DXR\t0.08486964232560668\t1970-01-03T00:06:00.000000Z\n" +
"HBC\t0.0367581207471136\t1970-01-03T00:54:00.000000Z\n" +
"HBC\t0.7905675319675964\t1970-01-03T00:24:00.000000Z\n" +
"HBC\t0.6508594025855301\t1970-01-03T00:18:00.000000Z\n";
assertQuery(
"k\tj\tprice\tts\n",
"select sym k, sym2 j, price, ts from x where sym != 'ABB' and 2 = 1",
"k\tprice\tts\n",
"select sym k, price, ts from x where sym != 'ABB' order by k",
"create table x (\n" +
" sym symbol cache index,\n" +
" sym2 symbol cache index,\n" +
" price double,\n" +
" ts timestamp\n" +
") timestamp(ts) partition by DAY",
"ts",
null,
"insert into x select * from (select rnd_symbol('ABB', 'HBC', 'DXR') sym, \n" +
" rnd_symbol('D', 'E', 'F') sym2, \n" +
" rnd_double() price, \n" +
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(10)) timestamp (ts)",
" from long_sequence(10)) timestamp (ts)",
expected,
false
true
);
}
@Test
public void testNotEquals1SymbolsWithFilter() throws Exception {
public void testNotEquals1SymbolsWithInvariantOrderBy2SymsDesc() throws Exception {
final String expected = "k\tj\tprice\tts\n" +
"DXR\tF\t0.6778564558839208\t1970-01-03T00:48:00.000000Z\n" +
"DXR\tF\t0.299199045961845\t1970-01-03T00:06:00.000000Z\n" +
"DXR\tD\t0.38539947865244994\t1970-01-03T00:54:00.000000Z\n" +
"HBC\tE\t0.9856290845874263\t1970-01-03T00:18:00.000000Z\n" +
"HBC\tD\t0.7611029514995744\t1970-01-03T00:30:00.000000Z\n" +
"DXR\tF\t0.6778564558839208\t1970-01-03T00:48:00.000000Z\n";
"HBC\tD\t0.2390529010846525\t1970-01-03T00:42:00.000000Z\n" +
"HBC\tD\t0.7611029514995744\t1970-01-03T00:30:00.000000Z\n";
assertQuery(
"k\tj\tprice\tts\n",
"select sym k, sym2 j, price, ts from x where sym != 'ABB' and price > 0.5",
"select sym k, sym2 j, price, ts from x where sym != 'ABB' order by k, j desc",
"create table x (\n" +
" sym symbol cache index,\n" +
" sym2 symbol cache index,\n" +
" price double,\n" +
" ts timestamp\n" +
") timestamp(ts) partition by DAY",
"ts",
null,
"insert into x select * from (select rnd_symbol('ABB', 'HBC', 'DXR') sym, \n" +
" rnd_symbol('D', 'E', 'F') sym2, \n" +
" rnd_double() price, \n" +
......@@ -194,7 +196,9 @@ public class SymbolNotEqualsValueTest extends AbstractGriffinTest {
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(10)) timestamp (ts)",
expected,
false
false,
true,
true
);
}
......@@ -574,6 +578,9 @@ public class SymbolNotEqualsValueTest extends AbstractGriffinTest {
" timestamp_sequence(172800000000, 360000000) ts \n" +
" from long_sequence(300)) timestamp (ts)",
expected,
true,
true,
false,
true
);
}
......
......@@ -118,7 +118,8 @@ public class TruncateTest extends AbstractGriffinTest {
"10\n",
"select count() from x",
null,
false
false,
true
);
Assert.assertEquals(TRUNCATE, compiler.compile("truncate table x", sqlExecutionContext).getType());
......@@ -128,7 +129,8 @@ public class TruncateTest extends AbstractGriffinTest {
"0\n",
"select count() from x",
null,
false
false,
true
);
Assert.assertEquals(0, engine.getBusyWriterCount());
......@@ -153,7 +155,8 @@ public class TruncateTest extends AbstractGriffinTest {
"10\n",
"select count() from x",
null,
false
false,
true
);
Assert.assertEquals(TRUNCATE, compiler.compile("truncate table x;", sqlExecutionContext).getType());
......@@ -163,7 +166,8 @@ public class TruncateTest extends AbstractGriffinTest {
"0\n",
"select count() from x",
null,
false
false,
true
);
Assert.assertEquals(0, engine.getBusyWriterCount());
......@@ -187,7 +191,8 @@ public class TruncateTest extends AbstractGriffinTest {
"10\n",
"select count() from x",
null,
false
false,
true
);
assertQuery(
......@@ -195,7 +200,8 @@ public class TruncateTest extends AbstractGriffinTest {
"20\n",
"select count() from y",
null,
false
false,
true
);
CyclicBarrier useBarrier = new CyclicBarrier(2);
......@@ -231,7 +237,8 @@ public class TruncateTest extends AbstractGriffinTest {
"10\n",
"select count() from x",
null,
false
false,
true
);
assertQuery(
......@@ -239,7 +246,8 @@ public class TruncateTest extends AbstractGriffinTest {
"20\n",
"select count() from y",
null,
false
false,
true
);
haltLatch.await(1, TimeUnit.SECONDS);
......@@ -260,7 +268,8 @@ public class TruncateTest extends AbstractGriffinTest {
"10\n",
"select count() from x",
null,
false
false,
true
);
assertQuery(
......@@ -268,7 +277,8 @@ public class TruncateTest extends AbstractGriffinTest {
"20\n",
"select count() from y",
null,
false
false,
true
);
try {
......@@ -284,7 +294,8 @@ public class TruncateTest extends AbstractGriffinTest {
"10\n",
"select count() from x",
null,
false
false,
true
);
assertQuery(
......@@ -292,7 +303,8 @@ public class TruncateTest extends AbstractGriffinTest {
"20\n",
"select count() from y",
null,
false
false,
true
);
engine.releaseAllWriters();
......@@ -300,6 +312,34 @@ public class TruncateTest extends AbstractGriffinTest {
});
}
@Test
public void testTableOnly() throws Exception {
assertMemoryLeak(() -> {
createX();
assertQuery(
"count\n" +
"10\n",
"select count() from x",
null,
false,
true
);
Assert.assertEquals(TRUNCATE, compiler.compile("truncate table only x", sqlExecutionContext).getType());
assertQuery(
"count\n" +
"0\n",
"select count() from x",
null,
false,
true
);
});
}
@Test
public void testTruncateOpenReader() throws Exception {
TestUtils.assertMemoryLeak(() -> {
......@@ -310,7 +350,8 @@ public class TruncateTest extends AbstractGriffinTest {
"1000000\n",
"select count() from x",
null,
false
false,
true
);
try (RecordCursorFactory factory = compiler.compile("select * from x", sqlExecutionContext).getRecordCursorFactory()) {
......@@ -343,7 +384,8 @@ public class TruncateTest extends AbstractGriffinTest {
"10\n",
"select count() from x",
null,
false
false,
true
);
assertQuery(
......@@ -351,7 +393,8 @@ public class TruncateTest extends AbstractGriffinTest {
"20\n",
"select count() from y",
null,
false
false,
true
);
Assert.assertEquals(TRUNCATE, compiler.compile("truncate table x, y", sqlExecutionContext).getType());
......@@ -361,7 +404,8 @@ public class TruncateTest extends AbstractGriffinTest {
"0\n",
"select count() from x",
null,
false
false,
true
);
assertQuery(
......@@ -369,7 +413,8 @@ public class TruncateTest extends AbstractGriffinTest {
"0\n",
"select count() from y",
null,
false
false,
true
);
......@@ -378,32 +423,6 @@ public class TruncateTest extends AbstractGriffinTest {
});
}
@Test
public void testTableOnly() throws Exception {
assertMemoryLeak(() -> {
createX();
assertQuery(
"count\n" +
"10\n",
"select count() from x",
null,
false
);
Assert.assertEquals(TRUNCATE, compiler.compile("truncate table only x", sqlExecutionContext).getType());
assertQuery(
"count\n" +
"0\n",
"select count() from x",
null,
false
);
});
}
private void createX() throws SqlException {
createX(10);
}
......
......@@ -131,7 +131,7 @@ public class UnionTest extends AbstractGriffinTest {
try (RecordCursorFactory rcf = compiler.compile("x", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected, rcf, true, true);
assertCursor(expected, rcf, true, true, true);
}
SharedRandom.RANDOM.get().reset();
......@@ -161,7 +161,7 @@ public class UnionTest extends AbstractGriffinTest {
);
try (RecordCursorFactory factory = compiler.compile("select * from x union all y", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected2, factory, false, true);
assertCursor(expected2, factory, false, true, true);
}
});
}
......@@ -198,7 +198,7 @@ public class UnionTest extends AbstractGriffinTest {
);
try (RecordCursorFactory rcf = compiler.compile("x", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected, rcf, true, true);
assertCursor(expected, rcf, true, true, true);
}
SharedRandom.RANDOM.get().reset();
......@@ -222,7 +222,7 @@ public class UnionTest extends AbstractGriffinTest {
);
try (RecordCursorFactory factory = compiler.compile("select distinct t from x union all y union all z", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected2, factory, false, true);
assertCursor(expected2, factory, false, true, false);
}
});
}
......@@ -257,7 +257,7 @@ public class UnionTest extends AbstractGriffinTest {
);
try (RecordCursorFactory rcf = compiler.compile("x", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected, rcf, true, true);
assertCursor(expected, rcf, true, true, true);
}
SharedRandom.RANDOM.get().reset();
......@@ -272,7 +272,7 @@ public class UnionTest extends AbstractGriffinTest {
);
try (RecordCursorFactory factory = compiler.compile("select distinct t from x union all y", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected2, factory, false, true);
assertCursor(expected2, factory, false, true, false);
}
});
}
......@@ -413,7 +413,7 @@ public class UnionTest extends AbstractGriffinTest {
try (RecordCursorFactory rcf = compiler.compile("x", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected, rcf, true, true);
assertCursor(expected, rcf, true, true, true);
}
SharedRandom.RANDOM.get().reset();
......@@ -468,7 +468,7 @@ public class UnionTest extends AbstractGriffinTest {
);
try (RecordCursorFactory factory = compiler.compile("select * from x union y union z", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected2, factory, false, true);
assertCursor(expected2, factory, false, true, false);
}
});
}
......@@ -572,7 +572,7 @@ public class UnionTest extends AbstractGriffinTest {
try (RecordCursorFactory rcf = compiler.compile("x", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected, rcf, true, true);
assertCursor(expected, rcf, true, true, true);
}
SharedRandom.RANDOM.get().reset();
......@@ -629,7 +629,7 @@ public class UnionTest extends AbstractGriffinTest {
);
try (RecordCursorFactory factory = compiler.compile("select * from x union all y union z", sqlExecutionContext).getRecordCursorFactory()) {
assertCursor(expected2, factory, false, true);
assertCursor(expected2, factory, false, true, false);
}
});
}
......
......@@ -174,7 +174,8 @@ public class ClassCatalogueFunctionFactoryTest extends AbstractGriffinTest {
null,
false,
sqlExecutionContext,
false
false,
true
);
}
......@@ -205,7 +206,8 @@ public class ClassCatalogueFunctionFactoryTest extends AbstractGriffinTest {
null,
true,
sqlExecutionContext,
false
false,
true
);
}
}
\ No newline at end of file
......@@ -48,7 +48,9 @@ public class AvgDoubleVecGroupByFunctionFactoryTest extends AbstractGriffinTest
"alter table tab add column b double",
"avg\n" +
"0.511848387\n",
false
false,
true,
true
);
assertQuery(
......@@ -57,7 +59,9 @@ public class AvgDoubleVecGroupByFunctionFactoryTest extends AbstractGriffinTest
"select round(avg(f),6) avg, round(avg(b),6) avg2 from tab",
"insert into tab select rnd_double(2), rnd_double(2) from long_sequence(469)",
null,
false
false,
true,
true
);
}
......@@ -72,7 +76,9 @@ public class AvgDoubleVecGroupByFunctionFactoryTest extends AbstractGriffinTest
"insert into tab select 123 from long_sequence(1)",
"avg\n" +
"123.0\n",
false
false,
true,
true
);
}
......@@ -84,7 +90,9 @@ public class AvgDoubleVecGroupByFunctionFactoryTest extends AbstractGriffinTest
"select round(avg(f),9) avg from tab",
"create table tab as (select rnd_double(2) f from long_sequence(131))",
null,
false
false,
true,
true
);
}
}
\ No newline at end of file
......@@ -54,6 +54,8 @@ public class CountLong256GroupByFunctionFactoryTest extends AbstractGriffinTest
"select a, count(s) from x",
"create table x as (select * from (select rnd_symbol('a','b','c','d','e','f') a, rnd_long256(16) s, timestamp_sequence(0, 100000) ts from long_sequence(20)) timestamp(ts))",
null,
true,
true,
true
);
}
......@@ -66,7 +68,9 @@ public class CountLong256GroupByFunctionFactoryTest extends AbstractGriffinTest
"select count(s) from x",
"create table x as (select * from (select rnd_symbol('344', 'xx2', '00s', '544', 'rraa', '0llp') s, timestamp_sequence(0, 100000) ts from long_sequence(100)) timestamp(ts))",
null,
false
false,
true,
true
);
}
......@@ -87,6 +91,8 @@ public class CountLong256GroupByFunctionFactoryTest extends AbstractGriffinTest
"select ts, count(s) from x sample by 1s fill(linear)",
"create table x as (select * from (select rnd_long256(10) s, timestamp_sequence(0, 100000) ts from long_sequence(100)) timestamp(ts))",
"ts",
true,
true,
true
);
}
......
......@@ -54,6 +54,8 @@ public class CountStringGroupByFunctionFactoryTest extends AbstractGriffinTest {
"select a, count(s) from x",
"create table x as (select * from (select rnd_symbol('a','b','c','d','e','f') a, rnd_str('344', 'xx2', '00s', '544', 'rraa', '0llp') s, timestamp_sequence(0, 100000) ts from long_sequence(20)) timestamp(ts))",
null,
true,
true,
true
);
}
......@@ -66,7 +68,9 @@ public class CountStringGroupByFunctionFactoryTest extends AbstractGriffinTest {
"select count(s) from x",
"create table x as (select * from (select rnd_symbol('344', 'xx2', '00s', '544', 'rraa', '0llp') s, timestamp_sequence(0, 100000) ts from long_sequence(100)) timestamp(ts))",
null,
false
false,
true,
true
);
}
......@@ -87,6 +91,8 @@ public class CountStringGroupByFunctionFactoryTest extends AbstractGriffinTest {
"select ts, count(s) from x sample by 1s fill(linear)",
"create table x as (select * from (select rnd_str('344', 'xx2', '00s', '544', 'rraa', '0llp') s, timestamp_sequence(0, 100000) ts from long_sequence(100)) timestamp(ts))",
"ts",
true,
true,
true
);
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册